diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 606d4b590d65e..b6a0da7018612 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -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) TestChangingCharsetToUtf8(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/ddl/index.go b/ddl/index.go index e1aefee387c0f..842b0cb51f263 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -457,7 +457,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 @@ -543,8 +543,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) @@ -567,10 +567,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 { @@ -789,6 +798,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 diff --git a/tablecodec/tablecodec.go b/tablecodec/tablecodec.go index de9b31c31210e..7eeea8e57cb59 100644 --- a/tablecodec/tablecodec.go +++ b/tablecodec/tablecodec.go @@ -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 ( diff --git a/tablecodec/tablecodec_test.go b/tablecodec/tablecodec_test.go index a68f26d6ef873..9467d926128d9 100644 --- a/tablecodec/tablecodec_test.go +++ b/tablecodec/tablecodec_test.go @@ -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) { diff --git a/util/admin/admin.go b/util/admin/admin.go index 27838ca2cadc2..a09cfce31982c 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -570,7 +570,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)) @@ -595,7 +595,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 { diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index bd4d778b91b81..4ea12d1394299 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -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 { @@ -52,7 +52,7 @@ func NewRowDecoder(cols []*table.Column, decodeColMap map[int64]Column) RowDecod } } if !haveGenCol { - return RowDecoder{ + return &RowDecoder{ colTypes: colFieldMap, } } @@ -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, @@ -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)