Skip to content

Commit

Permalink
*: refactor the optional arguments for table.AddRecord and `index.C…
Browse files Browse the repository at this point in the history
…reate` (pingcap#11018)
  • Loading branch information
tiancaiamao authored Jul 2, 2019
1 parent cb23b52 commit ff82b62
Show file tree
Hide file tree
Showing 8 changed files with 67 additions and 16 deletions.
3 changes: 1 addition & 2 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2654,8 +2654,7 @@ func (s *testDBSuite4) TestAddColumn2(c *C) {
c.Assert(len(oldRow), Equals, 3)
err = writeOnlyTable.RemoveRecord(s.tk.Se, 1, oldRow)
c.Assert(err, IsNil)
_, err = writeOnlyTable.AddRecord(s.tk.Se, types.MakeDatums(oldRow[0].GetInt64(), 2, oldRow[2].GetInt64()),
&table.AddRecordOpt{IsUpdate: true})
_, err = writeOnlyTable.AddRecord(s.tk.Se, types.MakeDatums(oldRow[0].GetInt64(), 2, oldRow[2].GetInt64()), table.IsUpdate)
c.Assert(err, IsNil)
err = s.tk.Se.StmtCommit()
c.Assert(err, IsNil)
Expand Down
8 changes: 6 additions & 2 deletions executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,8 +141,12 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu
return false, false, 0, err
}
// the `affectedRows` is increased when adding new record.
newHandle, err = t.AddRecord(ctx, newData,
&table.AddRecordOpt{CreateIdxOpt: table.CreateIdxOpt{SkipHandleCheck: sc.DupKeyAsWarning}, IsUpdate: true})
if sc.DupKeyAsWarning {
newHandle, err = t.AddRecord(ctx, newData, table.IsUpdate)
} else {
newHandle, err = t.AddRecord(ctx, newData, table.IsUpdate, table.SkipHandleCheck)
}

if err != nil {
return false, false, 0, err
}
Expand Down
4 changes: 2 additions & 2 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1958,7 +1958,7 @@ func (it *infoschemaTable) RecordKey(h int64) kv.Key {
}

// AddRecord implements table.Table AddRecord interface.
func (it *infoschemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) {
func (it *infoschemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) {
return 0, table.ErrUnsupportedOp
}

Expand Down Expand Up @@ -2080,7 +2080,7 @@ func (vt *VirtualTable) RecordKey(h int64) kv.Key {
}

// AddRecord implements table.Table AddRecord interface.
func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) {
func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) {
return 0, table.ErrUnsupportedOp
}

Expand Down
17 changes: 16 additions & 1 deletion table/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,27 @@ type CreateIdxOpt struct {
SkipCheck bool // If true, skip all the unique indices constraint check.
}

// CreateIdxOptFunc is defined for the Create() method of Index interface.
// Here is a blog post about how to use this pattern:
// https://dave.cheney.net/2014/10/17/functional-options-for-friendly-apis
type CreateIdxOptFunc func(*CreateIdxOpt)

// SkipHandleCheck is a defined value of CreateIdxFunc.
var SkipHandleCheck CreateIdxOptFunc = func(opt *CreateIdxOpt) {
opt.SkipHandleCheck = true
}

// SkipCheck is a defined value of CreateIdxFunc.
var SkipCheck CreateIdxOptFunc = func(opt *CreateIdxOpt) {
opt.SkipCheck = true
}

// Index is the interface for index data on KV store.
type Index interface {
// Meta returns IndexInfo.
Meta() *model.IndexInfo
// Create supports insert into statement.
Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64, opts ...*CreateIdxOpt) (int64, error)
Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64, opts ...CreateIdxOpt) (int64, error)
// Delete supports delete from statement.
Delete(sc *stmtctx.StatementContext, m kv.Mutator, indexedValues []types.Datum, h int64, ss kv.Transaction) error
// Drop supports drop table, drop index statements.
Expand Down
22 changes: 21 additions & 1 deletion table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -96,6 +96,26 @@ type AddRecordOpt struct {
IsUpdate bool
}

// AddRecordOption is defined for the AddRecord() method of the Table interface.
type AddRecordOption interface {
ApplyOn(*AddRecordOpt)
}

// ApplyOn implements the AddRecordOption interface, so any CreateIdxOptFunc
// can be passed as the optional argument to the table.AddRecord method.
func (f CreateIdxOptFunc) ApplyOn(opt *AddRecordOpt) {
f(&opt.CreateIdxOpt)
}

// IsUpdate is a defined value for AddRecordOptFunc.
var IsUpdate AddRecordOption = isUpdate{}

type isUpdate struct{}

func (i isUpdate) ApplyOn(opt *AddRecordOpt) {
opt.IsUpdate = true
}

// Table is used to retrieve and modify rows in table.
type Table interface {
// IterRecords iterates records in the table and calls fn.
Expand Down Expand Up @@ -136,7 +156,7 @@ type Table interface {
RecordKey(h int64) kv.Key

// AddRecord inserts a row which should contain only public columns
AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*AddRecordOpt) (recordID int64, err error)
AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...AddRecordOption) (recordID int64, err error)

// UpdateRecord updates a row which should contain only writable columns.
UpdateRecord(ctx sessionctx.Context, h int64, currData, newData []types.Datum, touched []bool) error
Expand Down
2 changes: 1 addition & 1 deletion table/tables/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -197,7 +197,7 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.
// 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.
func (c *index) Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64,
opts ...*table.CreateIdxOpt) (int64, error) {
opts ...table.CreateIdxOpt) (int64, error) {
writeBufs := ctx.GetSessionVars().GetWriteStmtBufs()
skipCheck := ctx.GetSessionVars().LightningMode || ctx.GetSessionVars().StmtCtx.BatchCheck
key, distinct, err := c.GenIndexKey(ctx.GetSessionVars().StmtCtx, indexedValues, h, writeBufs.IndexKeyBuf)
Expand Down
2 changes: 1 addition & 1 deletion table/tables/partition.go
Original file line number Diff line number Diff line change
Expand Up @@ -331,7 +331,7 @@ func (t *partitionedTable) GetPartitionByRow(ctx sessionctx.Context, r []types.D
}

// AddRecord implements the AddRecord method for the table.Table interface.
func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) {
func (t *partitionedTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) {
partitionInfo := t.meta.GetPartitionInfo()
pid, err := t.locatePartition(ctx, partitionInfo, r)
if err != nil {
Expand Down
25 changes: 19 additions & 6 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -418,10 +418,10 @@ func (t *tableCommon) getRollbackableMemStore(ctx sessionctx.Context) (kv.Retrie
}

// AddRecord implements table.Table AddRecord interface.
func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) {
opt := &table.AddRecordOpt{}
if len(opts) != 0 {
opt = opts[0]
func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID int64, err error) {
var opt table.AddRecordOpt
for _, fn := range opts {
fn.ApplyOn(&opt)
}
var hasRecordID bool
cols := t.Cols()
Expand Down Expand Up @@ -456,8 +456,17 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ..
sessVars := ctx.GetSessionVars()

rm, err := t.getRollbackableMemStore(ctx)
var createIdxOpts []table.CreateIdxOptFunc
if len(opts) > 0 {
createIdxOpts = make([]table.CreateIdxOptFunc, 0, len(opts))
for _, fn := range opts {
if raw, ok := fn.(table.CreateIdxOptFunc); ok {
createIdxOpts = append(createIdxOpts, raw)
}
}
}
// Insert new entries into indices.
h, err := t.addIndices(ctx, recordID, r, rm, &opt.CreateIdxOpt)
h, err := t.addIndices(ctx, recordID, r, rm, createIdxOpts)
if err != nil {
return h, err
}
Expand Down Expand Up @@ -555,13 +564,17 @@ func (t *tableCommon) genIndexKeyStr(colVals []types.Datum) (string, error) {

// addIndices adds data into indices. If any key is duplicated, returns the original handle.
func (t *tableCommon) addIndices(ctx sessionctx.Context, recordID int64, r []types.Datum, rm kv.RetrieverMutator,
opt *table.CreateIdxOpt) (int64, error) {
opts []table.CreateIdxOptFunc) (int64, error) {
txn, err := ctx.Txn(true)
if err != nil {
return 0, err
}
// Clean up lazy check error environment
defer txn.DelOption(kv.PresumeKeyNotExistsError)
var opt table.CreateIdxOpt
for _, fn := range opts {
fn(&opt)
}
skipCheck := ctx.GetSessionVars().LightningMode || ctx.GetSessionVars().StmtCtx.BatchCheck
if t.meta.PKIsHandle && !skipCheck && !opt.SkipHandleCheck {
if err := CheckHandleExists(ctx, t, recordID, nil); err != nil {
Expand Down

0 comments on commit ff82b62

Please sign in to comment.