Skip to content

Commit

Permalink
ddl: fix panic when add index of generated column. (pingcap#8620)
Browse files Browse the repository at this point in the history
  • Loading branch information
winkyao authored and crazycs520 committed Dec 13, 2018
1 parent a221ae0 commit cf2d7e6
Show file tree
Hide file tree
Showing 6 changed files with 40 additions and 12 deletions.
18 changes: 18 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,24 @@ func (s *testIntegrationSuite) TestEndIncluded(c *C) {
tk.MustExec("admin check table t")
}

func (s *testIntegrationSuite) TestNullGeneratedColumn(c *C) {
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("CREATE TABLE `t` (" +
"`a` int(11) DEFAULT NULL," +
"`b` int(11) DEFAULT NULL," +
"`c` int(11) GENERATED ALWAYS AS (`a` + `b`) VIRTUAL DEFAULT NULL," +
"`h` varchar(10) DEFAULT NULL," +
"`m` int(11) DEFAULT NULL" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")

tk.MustExec("insert into t values()")
tk.MustExec("alter table t add index idx_c(c)")
tk.MustExec("drop table t")
}

func (s *testIntegrationSuite) TestCaseInsensitiveCharsetAndCollate(c *C) {
tk := testkit.NewTestKit(c, s.store)

Expand Down
16 changes: 13 additions & 3 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -456,7 +456,7 @@ type addIndexWorker struct {
defaultVals []types.Datum
idxRecords []*indexRecord
rowMap map[int64]types.Datum
rowDecoder decoder.RowDecoder
rowDecoder *decoder.RowDecoder
idxKeyBufs [][]byte
batchCheckKeys []kv.Key
distinctCheckFlags []bool
Expand Down Expand Up @@ -542,8 +542,8 @@ func (w *addIndexWorker) getIndexRecord(handle int64, recordKey []byte, rawRecor
}
continue
}
idxColumnVal := w.rowMap[col.ID]
if _, ok := w.rowMap[col.ID]; ok {
idxColumnVal, ok := w.rowMap[col.ID]
if ok {
idxVal[j] = idxColumnVal
// Make sure there is no dirty data.
delete(w.rowMap, col.ID)
Expand All @@ -566,10 +566,19 @@ func (w *addIndexWorker) getIndexRecord(handle int64, recordKey []byte, rawRecor
}
idxVal[j] = idxColumnVal
}
// If there are generated column, rowDecoder will use column value that not in idxInfo.Columns to calculate
// the generated value, so we need to clear up the reusing map.
w.cleanRowMap()
idxRecord := &indexRecord{handle: handle, key: recordKey, vals: idxVal}
return idxRecord, nil
}

func (w *addIndexWorker) cleanRowMap() {
for id := range w.rowMap {
delete(w.rowMap, id)
}
}

// getNextHandle gets next handle of entry that we are going to process.
func (w *addIndexWorker) getNextHandle(taskRange reorgIndexTask, taskDone bool) (nextHandle int64) {
if !taskDone {
Expand Down Expand Up @@ -788,6 +797,7 @@ func (w *addIndexWorker) handleBackfillTask(d *ddlCtx, task *reorgIndexTask) *ad
// we should check whether this ddl job is still runnable.
err = w.ddlWorker.isReorgRunnable(d)
}

if err != nil {
result.err = err
return result
Expand Down
4 changes: 2 additions & 2 deletions tablecodec/tablecodec.go
Original file line number Diff line number Diff line change
Expand Up @@ -305,10 +305,10 @@ func DecodeRowWithMap(b []byte, cols map[int64]*types.FieldType, loc *time.Locat
row = make(map[int64]types.Datum, len(cols))
}
if b == nil {
return nil, nil
return row, nil
}
if len(b) == 1 && b[0] == codec.NilFlag {
return nil, nil
return row, nil
}
cnt := 0
var (
Expand Down
2 changes: 1 addition & 1 deletion tablecodec/tablecodec_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,7 +137,7 @@ func (s *testTableCodecSuite) TestRowCodec(c *C) {

r, err = DecodeRow(bs, colMap, time.UTC)
c.Assert(err, IsNil)
c.Assert(r, IsNil)
c.Assert(len(r), Equals, 0)
}

func (s *testTableCodecSuite) TestTimeCodec(c *C) {
Expand Down
4 changes: 2 additions & 2 deletions util/admin/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -553,7 +553,7 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table.
return nil
}

func makeRowDecoder(t table.Table, decodeCol []*table.Column, genExpr map[model.TableColumnID]expression.Expression) decoder.RowDecoder {
func makeRowDecoder(t table.Table, decodeCol []*table.Column, genExpr map[model.TableColumnID]expression.Expression) *decoder.RowDecoder {
cols := t.Cols()
tblInfo := t.Meta()
decodeColsMap := make(map[int64]decoder.Column, len(decodeCol))
Expand All @@ -578,7 +578,7 @@ func makeRowDecoder(t table.Table, decodeCol []*table.Column, genExpr map[model.
}

// genExprs use to calculate generated column value.
func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h int64, cols []*table.Column, rowDecoder decoder.RowDecoder) ([]types.Datum, error) {
func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h int64, cols []*table.Column, rowDecoder *decoder.RowDecoder) ([]types.Datum, error) {
key := t.RecordKey(h)
value, err := txn.Get(key)
if err != nil {
Expand Down
8 changes: 4 additions & 4 deletions util/rowDecoder/decoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,7 +42,7 @@ type RowDecoder struct {
}

// NewRowDecoder returns a new RowDecoder.
func NewRowDecoder(cols []*table.Column, decodeColMap map[int64]Column) RowDecoder {
func NewRowDecoder(cols []*table.Column, decodeColMap map[int64]Column) *RowDecoder {
colFieldMap := make(map[int64]*types.FieldType, len(decodeColMap))
haveGenCol := false
for id, col := range decodeColMap {
Expand All @@ -52,7 +52,7 @@ func NewRowDecoder(cols []*table.Column, decodeColMap map[int64]Column) RowDecod
}
}
if !haveGenCol {
return RowDecoder{
return &RowDecoder{
colTypes: colFieldMap,
}
}
Expand All @@ -61,7 +61,7 @@ func NewRowDecoder(cols []*table.Column, decodeColMap map[int64]Column) RowDecod
for _, col := range cols {
tps[col.Offset] = &col.FieldType
}
return RowDecoder{
return &RowDecoder{
mutRow: chunk.MutRowFromTypes(tps),
columns: decodeColMap,
colTypes: colFieldMap,
Expand All @@ -70,7 +70,7 @@ func NewRowDecoder(cols []*table.Column, decodeColMap map[int64]Column) RowDecod
}

// DecodeAndEvalRowWithMap decodes a byte slice into datums and evaluates the generated column value.
func (rd RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) {
func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) {
row, err := tablecodec.DecodeRowWithMap(b, rd.colTypes, decodeLoc, row)
if err != nil {
return nil, errors.Trace(err)
Expand Down

0 comments on commit cf2d7e6

Please sign in to comment.