// Copyright 2020 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, // See the License for the specific language governing permissions and // limitations under the License. package session import ( "bytes" "context" "encoding/hex" "fmt" "reflect" "github.com/pingcap/errors" pb "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/rowcodec" "go.uber.org/zap" ) const amendableType = nonMemAmendType | memBufAmendType const nonMemAmendType = (1 << model.ActionAddColumn) | (1 << model.ActionDropColumn) | (1 << model.ActionDropIndex) const memBufAmendType = uint64(1< 0 } func needCollectIndexOps(actionType uint64) bool { return actionType&(1< idxInfoAtStart.Meta().State { amendOpType = ConstOpAddIndex[idxInfoAtStart.Meta().State][idxInfoAtCommit.Meta().State] } if amendOpType != AmendNone { opInfo := &amendOperationAddIndexInfo{} opInfo.AmendOpType = amendOpType opInfo.tblInfoAtStart = tblAtStart opInfo.tblInfoAtCommit = tblAtCommit opInfo.indexInfoAtStart = idxInfoAtStart opInfo.indexInfoAtCommit = idxInfoAtCommit for _, idxCol := range idxInfoAtCommit.Meta().Columns { colID := tblAtCommit.Meta().Columns[idxCol.Offset].ID oldColInfo := findColByID(tblAtStart, colID) // TODO: now index column MUST be found in old table columns, generated column is not supported. if oldColInfo == nil || oldColInfo.IsGenerated() || oldColInfo.Hidden { return nil, errors.Trace(errors.Errorf("amend index column=%v id=%v is not found or generated in table=%v", idxCol.Name, colID, tblAtCommit.Meta().Name.String())) } opInfo.relatedOldIdxCols = append(opInfo.relatedOldIdxCols, oldColInfo) } opInfo.schemaAndDecoder = newSchemaAndDecoder(sctx, tblAtStart.Meta()) fieldTypes := make([]*types.FieldType, 0, len(tblAtStart.Meta().Columns)) for _, col := range tblAtStart.Meta().Columns { fieldTypes = append(fieldTypes, &col.FieldType) } opInfo.chk = chunk.NewChunkWithCapacity(fieldTypes, 4) addNewIndexOp := &amendOperationAddIndex{ info: opInfo, insertedNewIndexKeys: make(map[string]struct{}), deletedOldIndexKeys: make(map[string]struct{}), } res = append(res, addNewIndexOp) } } return res, nil } // collectTblAmendOps collects amend operations for each table using the schema diff between startTS and commitTS. func (a *amendCollector) collectTblAmendOps(sctx sessionctx.Context, phyTblID int64, tblInfoAtStart, tblInfoAtCommit table.Table, actionType uint64) error { if _, ok := a.tblAmendOpMap[phyTblID]; !ok { a.tblAmendOpMap[phyTblID] = make([]amendOp, 0, 4) } if needCollectModifyColOps(actionType) { _, err := a.collectModifyColAmendOps(tblInfoAtStart, tblInfoAtCommit) if err != nil { return err } } if needCollectIndexOps(actionType) { // TODO: currently only "add index" is considered. ops, err := a.collectIndexAmendOps(sctx, tblInfoAtStart, tblInfoAtCommit) if err != nil { return err } a.tblAmendOpMap[phyTblID] = append(a.tblAmendOpMap[phyTblID], ops...) } return nil } // mayGenDelIndexRowKeyOp returns if the row key op could generate Op_Del index key mutations. func mayGenDelIndexRowKeyOp(keyOp pb.Op) bool { return keyOp == pb.Op_Del || keyOp == pb.Op_Put } // mayGenPutIndexRowKeyOp returns if the row key op could generate Op_Put/Op_Insert index key mutations. func mayGenPutIndexRowKeyOp(keyOp pb.Op) bool { return keyOp == pb.Op_Put || keyOp == pb.Op_Insert } // amendOp is an amend operation for a specific schema change, new mutations will be generated using input ones. type amendOp interface { genMutations(ctx context.Context, sctx sessionctx.Context, commitMutations tikv.CommitterMutations, kvMap *rowKvMap, resultMutations *tikv.CommitterMutations) error } // amendOperationAddIndex represents one amend operation related to a specific add index change. type amendOperationAddIndexInfo struct { AmendOpType int tblInfoAtStart table.Table tblInfoAtCommit table.Table indexInfoAtStart table.Index indexInfoAtCommit table.Index relatedOldIdxCols []*table.Column schemaAndDecoder *schemaAndDecoder chk *chunk.Chunk } // amendOperationAddIndex represents the add operation will be performed on new key values for add index amend. type amendOperationAddIndex struct { info *amendOperationAddIndexInfo // insertedNewIndexKeys is used to check duplicates for new index generated by unique key. insertedNewIndexKeys map[string]struct{} // deletedOldIndexKeys is used to check duplicates for deleted old index keys. deletedOldIndexKeys map[string]struct{} } func (a *amendOperationAddIndexInfo) String() string { var colStr string colStr += "[" for _, colInfo := range a.relatedOldIdxCols { colStr += fmt.Sprintf(" %s ", colInfo.Name) } colStr += "]" res := fmt.Sprintf("AmenedOpType=%d phyTblID=%d idxID=%d columns=%v", a.AmendOpType, a.indexInfoAtCommit.Meta().ID, a.indexInfoAtCommit.Meta().ID, colStr) return res } func (a *amendOperationAddIndex) genMutations(ctx context.Context, sctx sessionctx.Context, commitMutations tikv.CommitterMutations, kvMap *rowKvMap, resAddMutations *tikv.CommitterMutations) error { // There should be no duplicate keys in deletedOldIndexKeys and insertedNewIndexKeys. deletedMutations := tikv.NewCommiterMutations(32) insertedMutations := tikv.NewCommiterMutations(32) for i, key := range commitMutations.GetKeys() { if tablecodec.IsIndexKey(key) || tablecodec.DecodeTableID(key) != a.info.tblInfoAtCommit.Meta().ID { continue } var newIdxMutation *tikv.Mutation var oldIdxMutation *tikv.Mutation var err error keyOp := commitMutations.GetOps()[i] if addIndexNeedRemoveOp(a.info.AmendOpType) { if mayGenDelIndexRowKeyOp(keyOp) { oldIdxMutation, err = a.genOldIdxKey(ctx, sctx, key, kvMap.oldRowKvMap) if err != nil { return err } } } if addIndexNeedAddOp(a.info.AmendOpType) { if mayGenPutIndexRowKeyOp(keyOp) { newIdxMutation, err = a.genNewIdxKey(ctx, sctx, key, kvMap.newRowKvMap) if err != nil { return err } } } skipMerge := false if a.info.AmendOpType == AmendNeedAddDeleteAndInsert { // If the old index key is the same with new index key, then the index related row value // is not changed in this row, we don't need to add or remove index keys for this row. if oldIdxMutation != nil && newIdxMutation != nil { if bytes.Equal(oldIdxMutation.Key, newIdxMutation.Key) { skipMerge = true } } } if !skipMerge { if oldIdxMutation != nil { deletedMutations.AppendMutation(*oldIdxMutation) } if newIdxMutation != nil { insertedMutations.AppendMutation(*newIdxMutation) } } } // For unique index, there may be conflicts on the same unique index key from different rows.Consider a update statement, // "Op_Del" on row_key = 3, row_val = 4, the "Op_Del" unique_key_4 -> nil will be generated. // "Op_Put" on row_key = 0, row_val = 4, the "Op_Insert" unique_key_4 -> 0 will be generated. // The "Op_Insert" should cover the "Op_Del" otherwise the new put row value will not have a correspond index value. if a.info.indexInfoAtCommit.Meta().Unique { for i := 0; i < len(deletedMutations.GetKeys()); i++ { key := deletedMutations.GetKeys()[i] if _, ok := a.insertedNewIndexKeys[string(key)]; !ok { resAddMutations.Push(deletedMutations.GetOps()[i], key, deletedMutations.GetValues()[i], deletedMutations.GetPessimisticFlags()[i]) } } for i := 0; i < len(insertedMutations.GetKeys()); i++ { key := insertedMutations.GetKeys()[i] destKeyOp := pb.Op_Insert if _, ok := a.deletedOldIndexKeys[string(key)]; ok { destKeyOp = pb.Op_Put } resAddMutations.Push(destKeyOp, key, insertedMutations.GetValues()[i], insertedMutations.GetPessimisticFlags()[i]) } } else { resAddMutations.MergeMutations(deletedMutations) resAddMutations.MergeMutations(insertedMutations) } return nil } func (a *amendOperationAddIndexInfo) genIndexKeyValue(ctx context.Context, sctx sessionctx.Context, kvMap map[string][]byte, key []byte, kvHandle int64, keyOnly bool) ([]byte, []byte, error) { chk := a.chk chk.Reset() val, ok := kvMap[string(key)] if !ok { // The Op_Put may not exist in old value kv map. if keyOnly { return nil, nil, nil } return nil, nil, errors.Errorf("key=%v is not found in new row kv map", kv.Key(key).String()) } err := executor.DecodeRowValToChunk(sctx, a.schemaAndDecoder.schema, a.tblInfoAtStart.Meta(), kvHandle, val, chk, a.schemaAndDecoder.decoder) if err != nil { logutil.Logger(ctx).Warn("amend decode value to chunk failed", zap.Error(err)) return nil, nil, errors.Trace(err) } idxVals := make([]types.Datum, 0, len(a.indexInfoAtCommit.Meta().Columns)) for _, oldCol := range a.relatedOldIdxCols { idxVals = append(idxVals, chk.GetRow(0).GetDatum(oldCol.Offset, &oldCol.FieldType)) } // 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) if err != nil { logutil.Logger(ctx).Warn("amend generate index key failed", zap.Error(err)) return nil, nil, errors.Trace(err) } if keyOnly { return newIdxKey, []byte{}, nil } // 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) if err != nil { logutil.Logger(ctx).Warn("amend generate index values failed", zap.Error(err)) return nil, nil, errors.Trace(err) } return newIdxKey, newIdxVal, nil } func (a *amendOperationAddIndex) genNewIdxKey(ctx context.Context, sctx sessionctx.Context, key []byte, kvMap map[string][]byte) (*tikv.Mutation, error) { kvHandle, err := tablecodec.DecodeRowKey(key) if err != nil { logutil.Logger(ctx).Error("decode key error", zap.String("key", hex.EncodeToString(key)), zap.Error(err)) return nil, errors.Trace(err) } newIdxKey, newIdxValue, err := a.info.genIndexKeyValue(ctx, sctx, kvMap, key, kvHandle, false) if err != nil { return nil, errors.Trace(err) } newIndexOp := pb.Op_Put isPessimisticLock := false if _, ok := a.insertedNewIndexKeys[string(newIdxKey)]; ok { return nil, errors.Trace(errors.Errorf("amend process key same key=%v found for index=%v in table=%v", newIdxKey, a.info.indexInfoAtCommit.Meta().Name, a.info.tblInfoAtCommit.Meta().Name)) } if a.info.indexInfoAtCommit.Meta().Unique { newIndexOp = pb.Op_Insert isPessimisticLock = true } a.insertedNewIndexKeys[string(newIdxKey)] = struct{}{} newMutation := &tikv.Mutation{KeyOp: newIndexOp, Key: newIdxKey, Value: newIdxValue, IsPessimisticLock: isPessimisticLock} return newMutation, nil } func (a *amendOperationAddIndex) genOldIdxKey(ctx context.Context, sctx sessionctx.Context, key []byte, oldValKvMap map[string][]byte) (*tikv.Mutation, error) { kvHandle, err := tablecodec.DecodeRowKey(key) if err != nil { logutil.Logger(ctx).Error("decode key error", zap.String("key", hex.EncodeToString(key)), zap.Error(err)) return nil, errors.Trace(err) } // Generated delete index key value. newIdxKey, emptyVal, err := a.info.genIndexKeyValue(ctx, sctx, oldValKvMap, key, kvHandle, true) if err != nil { return nil, errors.Trace(err) } // For Op_Put the key may not exist in old key value map. if len(newIdxKey) > 0 { isPessimisticLock := false if _, ok := a.deletedOldIndexKeys[string(newIdxKey)]; ok { return nil, errors.Trace(errors.Errorf("amend process key same key=%v found for index=%v in table=%v", newIdxKey, a.info.indexInfoAtCommit.Meta().Name, a.info.tblInfoAtCommit.Meta().Name)) } if a.info.indexInfoAtCommit.Meta().Unique { isPessimisticLock = true } a.deletedOldIndexKeys[string(newIdxKey)] = struct{}{} return &tikv.Mutation{KeyOp: pb.Op_Del, Key: newIdxKey, Value: emptyVal, IsPessimisticLock: isPessimisticLock}, nil } return nil, nil } // SchemaAmender is used to amend pessimistic transactions for schema change. type SchemaAmender struct { sess *session } // NewSchemaAmenderForTikvTxn creates a schema amender for tikvTxn type. func NewSchemaAmenderForTikvTxn(sess *session) *SchemaAmender { amender := &SchemaAmender{sess: sess} return amender } func (s *SchemaAmender) getAmendableKeys(commitMutations tikv.CommitterMutations, info *amendCollector) ([]kv.Key, []kv.Key) { addKeys := make([]kv.Key, 0, len(commitMutations.GetKeys())) removeKeys := make([]kv.Key, 0, len(commitMutations.GetKeys())) for i, byteKey := range commitMutations.GetKeys() { if tablecodec.IsIndexKey(byteKey) || !info.keyHasAmendOp(byteKey) { continue } keyOp := commitMutations.GetOps()[i] if pb.Op_Put == keyOp { addKeys = append(addKeys, byteKey) removeKeys = append(removeKeys, byteKey) } else if pb.Op_Insert == keyOp { addKeys = append(addKeys, byteKey) } else if pb.Op_Del == keyOp { removeKeys = append(removeKeys, byteKey) } else { // Do nothing. } } return addKeys, removeKeys } type rowKvMap struct { oldRowKvMap map[string][]byte newRowKvMap map[string][]byte } func (s *SchemaAmender) prepareKvMap(ctx context.Context, commitMutations tikv.CommitterMutations, info *amendCollector) (*rowKvMap, error) { // Get keys need to be considered for the amend operation, currently only row keys. addKeys, removeKeys := s.getAmendableKeys(commitMutations, info) // BatchGet the new key values, the Op_Put and Op_Insert type keys in memory buffer. txn, err := s.sess.Txn(true) if err != nil { return nil, errors.Trace(err) } newValKvMap, err := txn.BatchGet(ctx, addKeys) if err != nil { logutil.Logger(ctx).Warn("amend failed to batch get kv new keys", zap.Error(err)) return nil, errors.Trace(err) } if len(newValKvMap) != len(addKeys) { logutil.Logger(ctx).Error("amend failed to batch get results invalid", zap.Int("addKeys len", len(addKeys)), zap.Int("newValKvMap", len(newValKvMap))) return nil, errors.Errorf("add keys has %v values but result kvMap has %v", len(addKeys), len(newValKvMap)) } // BatchGet the old key values, the Op_Del and Op_Put types keys in storage using forUpdateTS, the Op_put type is for // row update using the same row key, it may not exist. snapshot, err := s.sess.GetStore().GetSnapshot(kv.Version{Ver: s.sess.sessionVars.TxnCtx.GetForUpdateTS()}) if err != nil { logutil.Logger(ctx).Warn("amend failed to get snapshot using forUpdateTS", zap.Error(err)) return nil, errors.Trace(err) } oldValKvMap, err := snapshot.BatchGet(ctx, removeKeys) if err != nil { logutil.Logger(ctx).Warn("amend failed to batch get kv old keys", zap.Error(err)) return nil, errors.Trace(err) } res := &rowKvMap{ oldRowKvMap: oldValKvMap, newRowKvMap: newValKvMap, } return res, nil } func (s *SchemaAmender) checkDupKeys(ctx context.Context, mutations *tikv.CommitterMutations) error { // Check if there are duplicate key entries. checkMap := make(map[string]pb.Op) for i := 0; i < len(mutations.GetKeys()); i++ { key := mutations.GetKeys()[i] keyOp := mutations.GetOps()[i] keyVal := mutations.GetValues()[i] if foundOp, ok := checkMap[string(key)]; ok { logutil.Logger(ctx).Error("duplicate key found in amend result mutations", zap.Stringer("key", kv.Key(key)), zap.Stringer("foundKeyOp", foundOp), zap.Stringer("thisKeyOp", keyOp), zap.Stringer("thisKeyValue", kv.Key(keyVal))) return errors.Trace(errors.Errorf("duplicate key=%s is found in mutations", kv.Key(key).String())) } checkMap[string(key)] = keyOp } return nil } // genAllAmendMutations generates CommitterMutations for all tables and related amend operations. func (s *SchemaAmender) genAllAmendMutations(ctx context.Context, commitMutations tikv.CommitterMutations, info *amendCollector) (*tikv.CommitterMutations, error) { rowKvMap, err := s.prepareKvMap(ctx, commitMutations, info) if err != nil { return nil, err } // Do generate add/remove mutations processing each key. resultNewMutations := tikv.NewCommiterMutations(32) for _, amendOps := range info.tblAmendOpMap { for _, curOp := range amendOps { err := curOp.genMutations(ctx, s.sess, commitMutations, rowKvMap, &resultNewMutations) if err != nil { return nil, err } } } err = s.checkDupKeys(ctx, &resultNewMutations) if err != nil { return nil, err } return &resultNewMutations, nil } // AmendTxn does check and generate amend mutations based on input infoSchema and mutations, mutations need to prewrite // are returned, the input commitMutations will not be changed. func (s *SchemaAmender) AmendTxn(ctx context.Context, startInfoSchema tikv.SchemaVer, change *tikv.RelatedSchemaChange, commitMutations tikv.CommitterMutations) (*tikv.CommitterMutations, error) { // Get info schema meta infoSchemaAtStart := startInfoSchema.(infoschema.InfoSchema) infoSchemaAtCheck := change.LatestInfoSchema.(infoschema.InfoSchema) // Collect amend operations for each table by physical table ID. var needAmendMem bool amendCollector := newAmendCollector() for i, tblID := range change.PhyTblIDS { actionType := change.ActionTypes[i] // Check amendable flags, return if not supported flags exist. if actionType&(^amendableType) != 0 { logutil.Logger(ctx).Info("amend action type not supported for txn", zap.Int64("tblID", tblID), zap.Uint64("actionType", actionType)) return nil, errors.Trace(table.ErrUnsupportedOp) } // Partition table is not supported now. tblInfoAtStart, ok := infoSchemaAtStart.TableByID(tblID) if !ok { return nil, errors.Trace(errors.Errorf("tableID=%d is not found in infoSchema", tblID)) } if tblInfoAtStart.Meta().Partition != nil { logutil.Logger(ctx).Info("Amend for partition table is not supported", zap.String("tableName", tblInfoAtStart.Meta().Name.String()), zap.Int64("tableID", tblID)) return nil, errors.Trace(table.ErrUnsupportedOp) } tblInfoAtCommit, ok := infoSchemaAtCheck.TableByID(tblID) if !ok { return nil, errors.Trace(errors.Errorf("tableID=%d is not found in infoSchema", tblID)) } if actionType&(memBufAmendType) != 0 { needAmendMem = true err := amendCollector.collectTblAmendOps(s.sess, tblID, tblInfoAtStart, tblInfoAtCommit, actionType) if err != nil { return nil, err } } } // After amend operations collect, generate related new mutations based on input commitMutations if needAmendMem { return s.genAllAmendMutations(ctx, commitMutations, amendCollector) } return nil, nil } func newSchemaAndDecoder(ctx sessionctx.Context, tbl *model.TableInfo) *schemaAndDecoder { schema := expression.NewSchema(make([]*expression.Column, 0, len(tbl.Columns))...) for _, col := range tbl.Columns { colExpr := &expression.Column{ RetType: &col.FieldType, ID: col.ID, } if col.IsGenerated() && !col.GeneratedStored { // This will not be used since generated column is rejected in collectIndexAmendOps. colExpr.VirtualExpr = &expression.Constant{} } schema.Append(colExpr) } return &schemaAndDecoder{schema, executor.NewRowDecoder(ctx, schema, tbl)} }