From 9a07a07e69d9801a72de44d64cf6cc04098323d8 Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 11 Dec 2018 18:08:18 +0800 Subject: [PATCH] ddl: fix panic when add index of generated column. (#8620) --- ddl/db_integration_test.go | 18 ++++++ ddl/index.go | 18 +++++- tablecodec/tablecodec.go | 4 +- tablecodec/tablecodec_test.go | 2 +- util/admin/admin.go | 29 +++++++++ util/rowDecoder/decoder.go | 112 ++++++++++++++++++++++++++++++++++ 6 files changed, 178 insertions(+), 5 deletions(-) create mode 100644 util/rowDecoder/decoder.go diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index effcb92e2d832..381fdffaed351 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 newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { store, err := mockstore.NewMockTikvStore() if err != nil { diff --git a/ddl/index.go b/ddl/index.go index 993258e7b1913..8df92e21b2220 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -455,6 +455,10 @@ type addIndexWorker struct { defaultVals []types.Datum idxRecords []*indexRecord rowMap map[int64]types.Datum +<<<<<<< HEAD +======= + rowDecoder *decoder.RowDecoder +>>>>>>> d3b92e028... ddl: fix panic when add index of generated column. (#8620) idxKeyBufs [][]byte batchCheckKeys []kv.Key distinctCheckFlags []bool @@ -538,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) @@ -563,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 { @@ -784,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 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 e686a367fd58b..82a155a9321c0 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -551,8 +551,37 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table. return nil } +<<<<<<< HEAD // genExprs use to calculate generated column value. func rowWithCols(sessCtx sessionctx.Context, txn kv.Retriever, t table.Table, h int64, cols []*table.Column, genExprs map[string]expression.Expression) ([]types.Datum, error) { +======= +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)) + for _, v := range decodeCol { + col := cols[v.Offset] + tpExpr := decoder.Column{ + Info: col.ToInfo(), + } + if col.IsGenerated() && !col.GeneratedStored { + for _, c := range cols { + if _, ok := col.Dependences[c.Name.L]; ok { + decodeColsMap[c.ID] = decoder.Column{ + Info: c.ToInfo(), + } + } + } + tpExpr.GenExpr = genExpr[model.TableColumnID{TableID: tblInfo.ID, ColumnID: col.ID}] + } + decodeColsMap[col.ID] = tpExpr + } + return decoder.NewRowDecoder(cols, decodeColsMap) +} + +// 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) { +>>>>>>> d3b92e028... ddl: fix panic when add index of generated column. (#8620) key := t.RecordKey(h) value, err := txn.Get(key) genColFlag := false diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go new file mode 100644 index 0000000000000..4ea12d1394299 --- /dev/null +++ b/util/rowDecoder/decoder.go @@ -0,0 +1,112 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package decoder + +import ( + "time" + + "github.com/pingcap/errors" + "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/table" + "github.com/pingcap/tidb/tablecodec" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/chunk" +) + +// Column contains the info and generated expr of column. +type Column struct { + Info *model.ColumnInfo + GenExpr expression.Expression +} + +// RowDecoder decodes a byte slice into datums and eval the generated column value. +type RowDecoder struct { + mutRow chunk.MutRow + columns map[int64]Column + colTypes map[int64]*types.FieldType + haveGenColumn bool +} + +// NewRowDecoder returns a new 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 { + colFieldMap[id] = &col.Info.FieldType + if col.GenExpr != nil { + haveGenCol = true + } + } + if !haveGenCol { + return &RowDecoder{ + colTypes: colFieldMap, + } + } + + tps := make([]*types.FieldType, len(cols)) + for _, col := range cols { + tps[col.Offset] = &col.FieldType + } + return &RowDecoder{ + mutRow: chunk.MutRowFromTypes(tps), + columns: decodeColMap, + colTypes: colFieldMap, + haveGenColumn: haveGenCol, + } +} + +// 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) { + row, err := tablecodec.DecodeRowWithMap(b, rd.colTypes, decodeLoc, row) + if err != nil { + return nil, errors.Trace(err) + } + if !rd.haveGenColumn { + return row, nil + } + + for id, v := range row { + rd.mutRow.SetValue(rd.columns[id].Info.Offset, v.GetValue()) + } + for id, col := range rd.columns { + if col.GenExpr == nil { + continue + } + // Eval the column value + val, err := col.GenExpr.Eval(rd.mutRow.ToRow()) + if err != nil { + return nil, errors.Trace(err) + } + val, err = table.CastValue(ctx, val, col.Info) + if err != nil { + return nil, errors.Trace(err) + } + + if val.Kind() == types.KindMysqlTime && sysLoc != time.UTC { + t := val.GetMysqlTime() + if t.Type == mysql.TypeTimestamp { + err := t.ConvertTimeZone(sysLoc, time.UTC) + if err != nil { + return nil, errors.Trace(err) + } + val.SetMysqlTime(t) + } + } + row[id] = val + } + return row, nil +}