Skip to content

Commit

Permalink
executor: fix admin check table bug when the column is nil and has a …
Browse files Browse the repository at this point in the history
…default value (pingcap#6142)
  • Loading branch information
winkyao authored Mar 27, 2018
1 parent c0697f5 commit defccc1
Show file tree
Hide file tree
Showing 4 changed files with 57 additions and 16 deletions.
17 changes: 17 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -212,3 +212,20 @@ func (s *testSuite) TestAdminRecoverIndex1(c *C) {
tk.MustExec("admin check index admin_test c2")
tk.MustExec("admin check index admin_test `primary`")
}

func (s *testSuite) TestAdminCheckTable(c *C) {
// test NULL value.
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
tk.MustExec(`CREATE TABLE test_null (
a int(11) NOT NULL,
c int(11) NOT NULL,
PRIMARY KEY (a, c),
KEY idx_a (a)
) ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin`)

tk.MustExec(`insert into test_null(a, c) values(2, 2);`)
tk.MustExec(`ALTER TABLE test_null ADD COLUMN b int NULL DEFAULT '1795454803' AFTER a;`)
tk.MustExec(`ALTER TABLE test_null add index b(b);`)
tk.MustExec("ADMIN CHECK TABLE test_null")
}
2 changes: 1 addition & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -386,7 +386,7 @@ func (e *CheckTableExec) NextChunk(ctx context.Context, chk *chunk.Chunk) error
}
for _, idx := range tb.Indices() {
txn := e.ctx.Txn()
err = admin.CompareIndexData(e.ctx.GetSessionVars().StmtCtx, txn, tb, idx)
err = admin.CompareIndexData(e.ctx, txn, tb, idx)
if err != nil {
return errors.Errorf("%v err:%v", t.Name, err)
}
Expand Down
42 changes: 33 additions & 9 deletions util/admin/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -245,16 +245,16 @@ func ScanIndexData(sc *stmtctx.StatementContext, txn kv.Transaction, kvIndex tab
// CompareIndexData compares index data one by one.
// It returns nil if the data from the index is equal to the data from the table columns,
// otherwise it returns an error with a different set of records.
func CompareIndexData(sc *stmtctx.StatementContext, txn kv.Transaction, t table.Table, idx table.Index) error {
err := checkIndexAndRecord(txn, t, idx)
func CompareIndexData(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error {
err := checkIndexAndRecord(sessCtx, txn, t, idx)
if err != nil {
return errors.Trace(err)
}

return CheckRecordAndIndex(sc, txn, t, idx)
return CheckRecordAndIndex(sessCtx, txn, t, idx)
}

func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx table.Index) error {
func checkIndexAndRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error {
it, err := idx.SeekFirst(txn)
if err != nil {
return errors.Trace(err)
Expand All @@ -274,7 +274,7 @@ func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx table.Index) err
return errors.Trace(err)
}

vals2, err := rowWithCols(txn, t, h, cols)
vals2, err := rowWithCols(sessCtx, txn, t, h, cols)
if kv.ErrNotExist.Equal(err) {
record := &RecordData{Handle: h, Values: vals1}
err = errDateNotEqual.Gen("index:%v != record:%v", record, nil)
Expand All @@ -293,14 +293,29 @@ func checkIndexAndRecord(txn kv.Transaction, t table.Table, idx table.Index) err
}

// CheckRecordAndIndex is exported for testing.
func CheckRecordAndIndex(sc *stmtctx.StatementContext, txn kv.Transaction, t table.Table, idx table.Index) error {
func CheckRecordAndIndex(sessCtx sessionctx.Context, txn kv.Transaction, t table.Table, idx table.Index) error {
sc := sessCtx.GetSessionVars().StmtCtx
cols := make([]*table.Column, len(idx.Meta().Columns))
for i, col := range idx.Meta().Columns {
cols[i] = t.Cols()[col.Offset]
}

startKey := t.RecordKey(0)
filterFunc := func(h1 int64, vals1 []types.Datum, cols []*table.Column) (bool, error) {
for i, val := range vals1 {
col := cols[i]
if val.IsNull() {
if mysql.HasNotNullFlag(col.Flag) {
return false, errors.New("Miss")
}
// NULL value is regarded as its default value.
colDefVal, err := table.GetColOriginDefaultValue(sessCtx, col.ToInfo())
if err != nil {
return false, errors.Trace(err)
}
vals1[i] = colDefVal
}
}
isExist, h2, err := idx.Exist(sc, txn, vals1, h1)
if kv.ErrKeyExists.Equal(err) {
record1 := &RecordData{Handle: h1, Values: vals1}
Expand Down Expand Up @@ -430,7 +445,7 @@ func CompareTableRecord(txn kv.Transaction, t table.Table, data []*RecordData, e
return nil
}

func rowWithCols(txn kv.Retriever, t table.Table, h int64, cols []*table.Column) ([]types.Datum, error) {
func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h int64, cols []*table.Column) ([]types.Datum, error) {
key := t.RecordKey(h)
value, err := txn.Get(key)
if err != nil {
Expand Down Expand Up @@ -471,8 +486,17 @@ func rowWithCols(txn kv.Retriever, t table.Table, h int64, cols []*table.Column)
continue
}
ri, ok := row[col.ID]
if !ok && mysql.HasNotNullFlag(col.Flag) {
return nil, errors.New("Miss")
if !ok {
if mysql.HasNotNullFlag(col.Flag) {
return nil, errors.New("Miss")
}
// NULL value is regarded as its default value.
colDefVal, err := table.GetColOriginDefaultValue(sessCtx, col.ToInfo())
if err != nil {
return nil, errors.Trace(err)
}
v[i] = colDefVal
continue
}
v[i] = ri
}
Expand Down
12 changes: 6 additions & 6 deletions util/admin/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -342,7 +342,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta
txn, err := s.store.Begin()
c.Assert(err, IsNil)
sc := &stmtctx.StatementContext{TimeZone: time.Local}
err = CompareIndexData(sc, txn, tb, idx)
err = CompareIndexData(ctx, txn, tb, idx)
c.Assert(err, IsNil)

idxNames := []string{idx.Meta().Name.L}
Expand All @@ -362,7 +362,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta

txn, err = s.store.Begin()
c.Assert(err, IsNil)
err = CompareIndexData(sc, txn, tb, idx)
err = CompareIndexData(ctx, txn, tb, idx)
c.Assert(err, NotNil)
record1 := &RecordData{Handle: int64(3), Values: types.MakeDatums(int64(30))}
diffMsg := newDiffRetError("index", record1, nil)
Expand All @@ -383,7 +383,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta

txn, err = s.store.Begin()
c.Assert(err, IsNil)
err = CompareIndexData(sc, txn, tb, idx)
err = CompareIndexData(ctx, txn, tb, idx)
c.Assert(err, NotNil)
record2 := &RecordData{Handle: int64(3), Values: types.MakeDatums(int64(31))}
diffMsg = newDiffRetError("index", record1, record2)
Expand All @@ -401,7 +401,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta

txn, err = s.store.Begin()
c.Assert(err, IsNil)
err = CheckRecordAndIndex(sc, txn, tb, idx)
err = CheckRecordAndIndex(ctx, txn, tb, idx)
c.Assert(err, NotNil)
record2 = &RecordData{Handle: int64(5), Values: types.MakeDatums(int64(30))}
diffMsg = newDiffRetError("index", record1, record2)
Expand All @@ -421,7 +421,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta

txn, err = s.store.Begin()
c.Assert(err, IsNil)
err = CompareIndexData(sc, txn, tb, idx)
err = CompareIndexData(ctx, txn, tb, idx)
c.Assert(err, NotNil)
record1 = &RecordData{Handle: int64(4), Values: types.MakeDatums(int64(40))}
diffMsg = newDiffRetError("index", record1, nil)
Expand All @@ -442,7 +442,7 @@ func (s *testSuite) testIndex(c *C, ctx sessionctx.Context, dbName string, tb ta

txn, err = s.store.Begin()
c.Assert(err, IsNil)
err = CompareIndexData(sc, txn, tb, idx)
err = CompareIndexData(ctx, txn, tb, idx)
c.Assert(err, NotNil)
diffMsg = newDiffRetError("index", nil, record1)
c.Assert(err.Error(), DeepEquals, diffMsg)
Expand Down

0 comments on commit defccc1

Please sign in to comment.