Skip to content

Commit

Permalink
ddl: fix panic when add index of generated column. (#8620) (#8676)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored and zimulala committed Dec 18, 2018
1 parent 9f6ec69 commit 2106f61
Show file tree
Hide file tree
Showing 12 changed files with 86 additions and 118 deletions.
2 changes: 1 addition & 1 deletion .travis.yml
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ language: go
go_import_path: github.com/pingcap/tidb

go:
- "1.11"
- "1.11.3"

env:
- TRAVIS_COVERAGE=0
Expand Down
2 changes: 1 addition & 1 deletion circle.yml
Original file line number Diff line number Diff line change
Expand Up @@ -3,7 +3,7 @@ version: 2
jobs:
build:
docker:
- image: golang:1.11
- image: golang:1.11.3
working_directory: /go/src/github.com/pingcap/tidb
steps:
- checkout
Expand Down
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
2 changes: 1 addition & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -410,7 +410,7 @@ type CheckTableExec struct {
done bool
is infoschema.InfoSchema

genExprs map[string]expression.Expression
genExprs map[model.TableColumnID]expression.Expression
}

// Open implements the Executor Open interface.
Expand Down
8 changes: 4 additions & 4 deletions model/model.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package model

import (
"encoding/json"
"fmt"
"strings"
"time"

Expand Down Expand Up @@ -556,7 +555,8 @@ func collationToProto(c string) int32 {
return int32(mysql.DefaultCollationID)
}

// GetTableColumnID gets a ID of a column with table ID
func GetTableColumnID(tableInfo *TableInfo, col *ColumnInfo) string {
return fmt.Sprintf("%d_%d", tableInfo.ID, col.ID)
// TableColumnID is composed by table ID and column ID.
type TableColumnID struct {
TableID int64
ColumnID int64
}
2 changes: 1 addition & 1 deletion planner/core/common_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ type CheckTable struct {

Tables []*ast.TableName

GenExprs map[string]expression.Expression
GenExprs map[model.TableColumnID]expression.Expression
}

// RecoverIndex is used for backfilling corrupted index data.
Expand Down
6 changes: 2 additions & 4 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,6 @@ import (
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/types/parser_driver"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/ranger"
)

Expand Down Expand Up @@ -530,7 +529,7 @@ func (b *planBuilder) buildAdmin(as *ast.AdminStmt) (Plan, error) {

func (b *planBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, error) {
p := &CheckTable{Tables: as.Tables}
p.GenExprs = make(map[string]expression.Expression)
p.GenExprs = make(map[model.TableColumnID]expression.Expression, len(p.Tables))

mockTablePlan := LogicalTableDual{}.init(b.ctx)
for _, tbl := range p.Tables {
Expand Down Expand Up @@ -562,8 +561,7 @@ func (b *planBuilder) buildAdminCheckTable(as *ast.AdminStmt) (*CheckTable, erro
return nil, errors.Trace(err)
}
expr = expression.BuildCastFunction(b.ctx, expr, colExpr.GetType())
genColumnName := admin.GetTableColumnID(tableInfo, column.ColumnInfo)
p.GenExprs[genColumnName] = expr
p.GenExprs[model.TableColumnID{TableID: tableInfo.ID, ColumnID: column.ColumnInfo.ID}] = expr
}
}
return p, nil
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
Loading

0 comments on commit 2106f61

Please sign in to comment.