Skip to content

Commit

Permalink
add an option for AddRecord and Create (#8884)
Browse files Browse the repository at this point in the history
  • Loading branch information
jackysp committed Jan 2, 2019
1 parent 61d2a1f commit e06c87d
Show file tree
Hide file tree
Showing 21 changed files with 90 additions and 73 deletions.
8 changes: 4 additions & 4 deletions ddl/column_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) {
// insert t values (1, 2);
originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID)
row := types.MakeDatums(1, 2)
h, err := originTable.AddRecord(ctx, row, false)
h, err := originTable.AddRecord(ctx, row)
c.Assert(err, IsNil)
txn, err := ctx.Txn(true)
c.Assert(err, IsNil)
Expand Down Expand Up @@ -180,7 +180,7 @@ func (s *testColumnChangeSuite) testAddColumnNoDefault(c *C, ctx sessionctx.Cont
if err != nil {
checkErr = errors.Trace(err)
}
_, err = writeOnlyTable.AddRecord(hookCtx, types.MakeDatums(10, 10), false)
_, err = writeOnlyTable.AddRecord(hookCtx, types.MakeDatums(10, 10))
if err != nil {
checkErr = errors.Trace(err)
}
Expand Down Expand Up @@ -235,7 +235,7 @@ func (s *testColumnChangeSuite) checkAddWriteOnly(ctx sessionctx.Context, d *ddl
if err != nil {
return errors.Trace(err)
}
_, err = writeOnlyTable.AddRecord(ctx, types.MakeDatums(2, 3), false)
_, err = writeOnlyTable.AddRecord(ctx, types.MakeDatums(2, 3))
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -310,7 +310,7 @@ func (s *testColumnChangeSuite) checkAddPublic(sctx sessionctx.Context, d *ddl,
if err != nil {
return errors.Trace(err)
}
h, err := publicTable.AddRecord(sctx, types.MakeDatums(4, 4, 4), false)
h, err := publicTable.AddRecord(sctx, types.MakeDatums(4, 4, 4))
if err != nil {
return errors.Trace(err)
}
Expand Down
16 changes: 8 additions & 8 deletions ddl/column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -124,7 +124,7 @@ func (s *testColumnSuite) TestColumn(c *C) {

num := 10
for i := 0; i < num; i++ {
_, err := t.AddRecord(ctx, types.MakeDatums(i, 10*i, 100*i), false)
_, err := t.AddRecord(ctx, types.MakeDatums(i, 10*i, 100*i))
c.Assert(err, IsNil)
}

Expand Down Expand Up @@ -164,7 +164,7 @@ func (s *testColumnSuite) TestColumn(c *C) {
c.Assert(err, IsNil)
c.Assert(i, Equals, int64(num))

h, err := t.AddRecord(ctx, types.MakeDatums(11, 12, 13, 14), false)
h, err := t.AddRecord(ctx, types.MakeDatums(11, 12, 13, 14))
c.Assert(err, IsNil)
err = ctx.NewTxn(context.Background())
c.Assert(err, IsNil)
Expand Down Expand Up @@ -359,7 +359,7 @@ func (s *testColumnSuite) checkDeleteOnlyColumn(ctx sessionctx.Context, d *ddl,
}

newRow := types.MakeDatums(int64(11), int64(22), int64(33))
newHandle, err := t.AddRecord(ctx, newRow, false)
newHandle, err := t.AddRecord(ctx, newRow)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -463,7 +463,7 @@ func (s *testColumnSuite) checkWriteOnlyColumn(ctx sessionctx.Context, d *ddl, t
}

newRow := types.MakeDatums(int64(11), int64(22), int64(33))
newHandle, err := t.AddRecord(ctx, newRow, false)
newHandle, err := t.AddRecord(ctx, newRow)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -563,7 +563,7 @@ func (s *testColumnSuite) checkReorganizationColumn(ctx sessionctx.Context, d *d
}

newRow := types.MakeDatums(int64(11), int64(22), int64(33))
newHandle, err := t.AddRecord(ctx, newRow, false)
newHandle, err := t.AddRecord(ctx, newRow)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -660,7 +660,7 @@ func (s *testColumnSuite) checkPublicColumn(ctx sessionctx.Context, d *ddl, tblI
}

newRow := types.MakeDatums(int64(11), int64(22), int64(33), int64(44))
handle, err = t.AddRecord(ctx, newRow, false)
handle, err = t.AddRecord(ctx, newRow)
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -765,7 +765,7 @@ func (s *testColumnSuite) TestAddColumn(c *C) {
t := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID)

oldRow := types.MakeDatums(int64(1), int64(2), int64(3))
handle, err := t.AddRecord(ctx, oldRow, false)
handle, err := t.AddRecord(ctx, oldRow)
c.Assert(err, IsNil)

txn, err := ctx.Txn(true)
Expand Down Expand Up @@ -856,7 +856,7 @@ func (s *testColumnSuite) TestDropColumn(c *C) {
colName := "c4"
defaultColValue := int64(4)
row := types.MakeDatums(int64(1), int64(2), int64(3))
_, err = t.AddRecord(ctx, append(row, types.NewDatum(defaultColValue)), false)
_, err = t.AddRecord(ctx, append(row, types.NewDatum(defaultColValue)))
c.Assert(err, IsNil)

txn, err := ctx.Txn(true)
Expand Down
14 changes: 7 additions & 7 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -436,7 +436,7 @@ func (s *testDDLSuite) TestCancelJob(c *C) {
// insert t values (1, 2);
originTable := testGetTable(c, d, dbInfo.ID, tblInfo.ID)
row := types.MakeDatums(1, 2)
_, err = originTable.AddRecord(ctx, row, false)
_, err = originTable.AddRecord(ctx, row)
c.Assert(err, IsNil)
txn, err := ctx.Txn(true)
c.Assert(err, IsNil)
Expand Down Expand Up @@ -709,9 +709,9 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
testCreateTable(c, ctx, d, dbInfo1, tblInfo1)
// insert t1 values (10, 10), (20, 20)
tbl1 := testGetTable(c, d, dbInfo1.ID, tblInfo1.ID)
_, err = tbl1.AddRecord(ctx, types.MakeDatums(1, 1), false)
_, err = tbl1.AddRecord(ctx, types.MakeDatums(1, 1))
c.Assert(err, IsNil)
_, err = tbl1.AddRecord(ctx, types.MakeDatums(2, 2), false)
_, err = tbl1.AddRecord(ctx, types.MakeDatums(2, 2))
c.Assert(err, IsNil)
// create table t2 (c1 int primary key, c2 int, c3 int);
tblInfo2 := testTableInfo(c, d, "t2", 3)
Expand All @@ -720,11 +720,11 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
testCreateTable(c, ctx, d, dbInfo1, tblInfo2)
// insert t2 values (1, 1), (2, 2), (3, 3)
tbl2 := testGetTable(c, d, dbInfo1.ID, tblInfo2.ID)
_, err = tbl2.AddRecord(ctx, types.MakeDatums(1, 1, 1), false)
_, err = tbl2.AddRecord(ctx, types.MakeDatums(1, 1, 1))
c.Assert(err, IsNil)
_, err = tbl2.AddRecord(ctx, types.MakeDatums(2, 2, 2), false)
_, err = tbl2.AddRecord(ctx, types.MakeDatums(2, 2, 2))
c.Assert(err, IsNil)
_, err = tbl2.AddRecord(ctx, types.MakeDatums(3, 3, 3), false)
_, err = tbl2.AddRecord(ctx, types.MakeDatums(3, 3, 3))
c.Assert(err, IsNil)
// create database test_parallel_ddl_2;
dbInfo2 := testSchemaInfo(c, d, "test_parallel_ddl_2")
Expand All @@ -734,7 +734,7 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
testCreateTable(c, ctx, d, dbInfo2, tblInfo3)
// insert t3 values (11, 22, 33, 44)
tbl3 := testGetTable(c, d, dbInfo2.ID, tblInfo3.ID)
_, err = tbl3.AddRecord(ctx, types.MakeDatums(11, 22, 33, 44), false)
_, err = tbl3.AddRecord(ctx, types.MakeDatums(11, 22, 33, 44))
c.Assert(err, IsNil)

// set hook to execute jobs after all jobs are in queue.
Expand Down
2 changes: 1 addition & 1 deletion ddl/fail_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ func (s *testColumnChangeSuite) TestFailBeforeDecodeArgs(c *C) {
// insert t_fail values (1, 2);
originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID)
row := types.MakeDatums(1, 2)
_, err = originTable.AddRecord(ctx, row, false)
_, err = originTable.AddRecord(ctx, row)
c.Assert(err, IsNil)
txn, err := ctx.Txn(true)
c.Assert(err, IsNil)
Expand Down
20 changes: 10 additions & 10 deletions ddl/index_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,11 +68,11 @@ func (s *testIndexChangeSuite) TestIndexChange(c *C) {
originTable := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID)

// insert t values (1, 1), (2, 2), (3, 3)
_, err = originTable.AddRecord(ctx, types.MakeDatums(1, 1), false)
_, err = originTable.AddRecord(ctx, types.MakeDatums(1, 1))
c.Assert(err, IsNil)
_, err = originTable.AddRecord(ctx, types.MakeDatums(2, 2), false)
_, err = originTable.AddRecord(ctx, types.MakeDatums(2, 2))
c.Assert(err, IsNil)
_, err = originTable.AddRecord(ctx, types.MakeDatums(3, 3), false)
_, err = originTable.AddRecord(ctx, types.MakeDatums(3, 3))
c.Assert(err, IsNil)

txn, err := ctx.Txn(true)
Expand Down Expand Up @@ -199,7 +199,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context,
if err != nil {
return errors.Trace(err)
}
_, err = delOnlyTbl.AddRecord(ctx, types.MakeDatums(4, 4), false)
_, err = delOnlyTbl.AddRecord(ctx, types.MakeDatums(4, 4))
if err != nil {
return errors.Trace(err)
}
Expand All @@ -209,7 +209,7 @@ func (s *testIndexChangeSuite) checkAddWriteOnly(d *ddl, ctx sessionctx.Context,
}

// WriteOnlyTable: insert t values (5, 5);
_, err = writeOnlyTbl.AddRecord(ctx, types.MakeDatums(5, 5), false)
_, err = writeOnlyTbl.AddRecord(ctx, types.MakeDatums(5, 5))
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -272,7 +272,7 @@ func (s *testIndexChangeSuite) checkAddPublic(d *ddl, ctx sessionctx.Context, wr
if err != nil {
return errors.Trace(err)
}
_, err = writeTbl.AddRecord(ctx, types.MakeDatums(6, 6), false)
_, err = writeTbl.AddRecord(ctx, types.MakeDatums(6, 6))
if err != nil {
return errors.Trace(err)
}
Expand All @@ -281,7 +281,7 @@ func (s *testIndexChangeSuite) checkAddPublic(d *ddl, ctx sessionctx.Context, wr
return errors.Trace(err)
}
// PublicTable: insert t values (7, 7)
_, err = publicTbl.AddRecord(ctx, types.MakeDatums(7, 7), false)
_, err = publicTbl.AddRecord(ctx, types.MakeDatums(7, 7))
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -343,7 +343,7 @@ func (s *testIndexChangeSuite) checkDropWriteOnly(d *ddl, ctx sessionctx.Context
if err != nil {
return errors.Trace(err)
}
_, err = writeTbl.AddRecord(ctx, types.MakeDatums(8, 8), false)
_, err = writeTbl.AddRecord(ctx, types.MakeDatums(8, 8))
if err != nil {
return errors.Trace(err)
}
Expand Down Expand Up @@ -387,7 +387,7 @@ func (s *testIndexChangeSuite) checkDropDeleteOnly(d *ddl, ctx sessionctx.Contex
if err != nil {
return errors.Trace(err)
}
_, err = writeTbl.AddRecord(ctx, types.MakeDatums(9, 9), false)
_, err = writeTbl.AddRecord(ctx, types.MakeDatums(9, 9))
if err != nil {
return errors.Trace(err)
}
Expand All @@ -398,7 +398,7 @@ func (s *testIndexChangeSuite) checkDropDeleteOnly(d *ddl, ctx sessionctx.Contex
}

// DeleteOnlyTable insert t values (10, 10)
_, err = delTbl.AddRecord(ctx, types.MakeDatums(10, 10), false)
_, err = delTbl.AddRecord(ctx, types.MakeDatums(10, 10))
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/reorg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ func (s *testDDLSuite) TestReorgOwner(c *C) {

num := 10
for i := 0; i < num; i++ {
_, err := t.AddRecord(ctx, types.MakeDatums(i, i, i), false)
_, err := t.AddRecord(ctx, types.MakeDatums(i, i, i))
c.Assert(err, IsNil)
}

Expand Down
4 changes: 2 additions & 2 deletions ddl/schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func (s *testSchemaSuite) TestSchema(c *C) {
testCheckJobDone(c, d, tJob1, true)
tbl1 := testGetTable(c, d, dbInfo.ID, tblInfo1.ID)
for i := 1; i <= 100; i++ {
_, err := tbl1.AddRecord(ctx, types.MakeDatums(i, i, i), false)
_, err := tbl1.AddRecord(ctx, types.MakeDatums(i, i, i))
c.Assert(err, IsNil)
}
// create table t1 with 1034 records.
Expand All @@ -156,7 +156,7 @@ func (s *testSchemaSuite) TestSchema(c *C) {
testCheckJobDone(c, d, tJob2, true)
tbl2 := testGetTable(c, d, dbInfo.ID, tblInfo2.ID)
for i := 1; i <= 1034; i++ {
_, err := tbl2.AddRecord(ctx, types.MakeDatums(i, i, i), false)
_, err := tbl2.AddRecord(ctx, types.MakeDatums(i, i, i))
c.Assert(err, IsNil)
}
job, v := testDropSchema(c, ctx, d, dbInfo)
Expand Down
2 changes: 1 addition & 1 deletion ddl/table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -277,7 +277,7 @@ func (s *testTableSuite) TestTable(c *C) {
count := 2000
tbl := testGetTable(c, d, s.dbInfo.ID, tblInfo.ID)
for i := 1; i <= count; i++ {
_, err := tbl.AddRecord(ctx, types.MakeDatums(i, i, i), false)
_, err := tbl.AddRecord(ctx, types.MakeDatums(i, i, i))
c.Assert(err, IsNil)
}

Expand Down
4 changes: 2 additions & 2 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2813,9 +2813,9 @@ func (s *testSuite) TestCheckIndex(c *C) {
recordVal1 := types.MakeDatums(int64(1), int64(10), int64(11))
recordVal2 := types.MakeDatums(int64(2), int64(20), int64(21))
c.Assert(s.ctx.NewTxn(context.Background()), IsNil)
_, err = tb.AddRecord(s.ctx, recordVal1, false)
_, err = tb.AddRecord(s.ctx, recordVal1)
c.Assert(err, IsNil)
_, err = tb.AddRecord(s.ctx, recordVal2, false)
_, err = tb.AddRecord(s.ctx, recordVal2)
c.Assert(err, IsNil)
txn, err := s.ctx.Txn(true)
c.Assert(err, IsNil)
Expand Down
2 changes: 1 addition & 1 deletion executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -585,7 +585,7 @@ func (e *InsertValues) addRecord(row []types.Datum) (int64, error) {
if !e.ctx.GetSessionVars().ConstraintCheckInPlace {
txn.SetOption(kv.PresumeKeyNotExists, nil)
}
h, err := e.Table.AddRecord(e.ctx, row, false)
h, err := e.Table.AddRecord(e.ctx, row)
txn.DelOption(kv.PresumeKeyNotExists)
if err != nil {
return 0, errors.Trace(err)
Expand Down
3 changes: 2 additions & 1 deletion executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,8 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu
return false, false, 0, errors.Trace(err)
}
// the `affectedRows` is increased when adding new record.
newHandle, err = t.AddRecord(ctx, newData, sc.DupKeyAsWarning)
newHandle, err = t.AddRecord(ctx, newData,
&table.AddRecordOpt{CreateIdxOpt: table.CreateIdxOpt{SkipHandleCheck: sc.DupKeyAsWarning}})
if err != nil {
return false, false, 0, errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/write_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2336,7 +2336,7 @@ func (s *testSuite2) TestRebaseIfNeeded(c *C) {
c.Assert(s.ctx.NewTxn(context.Background()), IsNil)
// AddRecord directly here will skip to rebase the auto ID in the insert statement,
// which could simulate another TiDB adds a large auto ID.
_, err = tbl.AddRecord(s.ctx, types.MakeDatums(30001, 2), false)
_, err = tbl.AddRecord(s.ctx, types.MakeDatums(30001, 2))
c.Assert(err, IsNil)
txn, err := s.ctx.Txn(true)
c.Assert(err, IsNil)
Expand Down
2 changes: 1 addition & 1 deletion infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1498,7 +1498,7 @@ func (it *infoschemaTable) RecordKey(h int64) kv.Key {
return nil
}

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

Expand Down
2 changes: 1 addition & 1 deletion perfschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -111,7 +111,7 @@ func (vt *perfSchemaTable) RecordKey(h int64) kv.Key {
}

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

Expand Down
8 changes: 7 additions & 1 deletion table/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,12 +27,18 @@ type IndexIterator interface {
Close()
}

// CreateIdxOpt contains the options will be used when creating an index.
type CreateIdxOpt struct {
SkipHandleCheck bool // If true, skip the handle constraint check.
SkipCheck bool // If true, skip all the unique indices constraint check.
}

// 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) (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) error
// Drop supports drop table, drop index statements.
Expand Down
8 changes: 6 additions & 2 deletions table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,6 +86,11 @@ var (
// RecordIterFunc is used for low-level record iteration.
type RecordIterFunc func(h int64, rec []types.Datum, cols []*Column) (more bool, err error)

// AddRecordOpt contains the options will be used when adding a record.
type AddRecordOpt struct {
CreateIdxOpt
}

// 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 @@ -126,8 +131,7 @@ type Table interface {
RecordKey(h int64) kv.Key

// AddRecord inserts a row which should contain only public columns
// skipHandleCheck indicates that recordID in r has been checked as not duplicate already.
AddRecord(ctx sessionctx.Context, r []types.Datum, skipHandleCheck bool) (recordID int64, err error)
AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*AddRecordOpt) (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
3 changes: 2 additions & 1 deletion table/tables/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -196,7 +196,8 @@ func (c *index) GenIndexKey(sc *stmtctx.StatementContext, indexedValues []types.
// Create creates a new entry in the kvIndex data.
// 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) (int64, error) {
func (c *index) Create(ctx sessionctx.Context, rm kv.RetrieverMutator, indexedValues []types.Datum, h int64,
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
Loading

0 comments on commit e06c87d

Please sign in to comment.