From 7267b96b0de84ee5ec3a4020a411adccb4c6945f Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Wed, 17 Jun 2020 15:32:49 +0800 Subject: [PATCH 01/10] done Signed-off-by: wjhuang2016 --- util/rowDecoder/decoder.go | 32 ++++++++++++++++++++++++++++++++ 1 file changed, 32 insertions(+) diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index c3678863a97fb..ee64485db36a0 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -27,6 +27,7 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" + "github.com/pingcap/tidb/util/codec" "github.com/pingcap/tidb/util/rowcodec" ) @@ -77,6 +78,30 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { } } +func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle) (bool, error) { + if handle == nil { + return false, nil + } + colInfo := dCol.Col.ColumnInfo + if dCol.Col.IsPKHandleColumn(rd.tbl.Meta()) { + if mysql.HasUnsignedFlag(colInfo.Flag) { + rd.mutRow.SetValue(colInfo.Offset, uint64(handle.IntValue())) + } else { + rd.mutRow.SetValue(colInfo.Offset, handle.IntValue()) + } + return true, nil + } + if !handle.IsInt() && mysql.HasPriKeyFlag(dCol.Col.Flag) { + _, d, err := codec.DecodeOne(handle.EncodedCol(colInfo.Offset)) + if err != nil { + return false, errors.Trace(err) + } + rd.mutRow.SetValue(colInfo.Offset, d) + return true, nil + } + return false, nil +} + // DecodeAndEvalRowWithMap decodes a byte slice into datums and evaluates the generated column value. func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { var err error @@ -99,6 +124,13 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) continue } + ok, err := rd.tryDecodeFromHandle(dCol, handle) + if err != nil { + return nil, err + } + if ok { + continue + } // Get the default value of the column in the generated column expression. val, err = tables.GetColDefaultValue(ctx, dCol.Col, rd.defaultVals) if err != nil { From b8210cc0dc453bbc86efe3d2b48a5634167d078d Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Thu, 18 Jun 2020 12:26:26 +0800 Subject: [PATCH 02/10] save Signed-off-by: wjhuang2016 --- ddl/index.go | 10 +-------- util/admin/admin.go | 2 +- util/rowDecoder/decoder.go | 40 +++++++++++++++++++++++---------- util/rowDecoder/decoder_test.go | 4 ++-- 4 files changed, 32 insertions(+), 24 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 8235e05b6b3f9..1be4900a46eb8 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -875,21 +875,13 @@ func (w *addIndexWorker) getIndexRecord(handle kv.Handle, recordKey []byte, rawR cols := t.Cols() idxInfo := w.index.Meta() sysZone := timeutil.SystemLocation() - _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, time.UTC, sysZone, w.rowMap) + _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, time.UTC, sysZone, w.rowMap, t.Meta()) if err != nil { return nil, errors.Trace(errCantDecodeIndex.GenWithStackByArgs(err)) } idxVal := make([]types.Datum, len(idxInfo.Columns)) for j, v := range idxInfo.Columns { col := cols[v.Offset] - if col.IsPKHandleColumn(t.Meta()) { - if mysql.HasUnsignedFlag(col.Flag) { - idxVal[j].SetUint64(uint64(handle.IntValue())) - } else { - idxVal[j].SetInt64(handle.IntValue()) - } - continue - } idxColumnVal, ok := w.rowMap[col.ID] if ok { idxVal[j] = idxColumnVal diff --git a/util/admin/admin.go b/util/admin/admin.go index 5638128fbc7fd..b125b3d85f600 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -438,7 +438,7 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab return errors.Trace(err) } - rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), time.UTC, nil) + rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), time.UTC, nil, t.Meta()) if err != nil { return errors.Trace(err) } diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index ee64485db36a0..47014041b6d9f 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -14,6 +14,8 @@ package decoder import ( + "github.com/pingcap/parser/model" + log "github.com/sirupsen/logrus" "sort" "time" @@ -78,7 +80,7 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { } } -func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle) (bool, error) { +func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle, pkCols []int64) (bool, error) { if handle == nil { return false, nil } @@ -91,19 +93,24 @@ func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle) (bool, } return true, nil } - if !handle.IsInt() && mysql.HasPriKeyFlag(dCol.Col.Flag) { - _, d, err := codec.DecodeOne(handle.EncodedCol(colInfo.Offset)) - if err != nil { - return false, errors.Trace(err) + // Try to decode common handle. + if mysql.HasPriKeyFlag(dCol.Col.Flag) { + for i, hid := range pkCols { + if dCol.Col.ID == hid { + _, d, err := codec.DecodeOne(handle.EncodedCol(i)) + if err != nil { + return false, errors.Trace(err) + } + rd.mutRow.SetValue(colInfo.Offset, d) + return true, nil + } } - rd.mutRow.SetValue(colInfo.Offset, d) - return true, nil } return false, nil } // DecodeAndEvalRowWithMap decodes a byte slice into datums and evaluates the generated column value. -func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { +func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum, tbl *model.TableInfo) (map[int64]types.Datum, error) { var err error if rowcodec.IsNewFormat(b) { row, err = tablecodec.DecodeRowWithMapNew(b, rd.colTypes, decodeLoc, row) @@ -113,22 +120,31 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. if err != nil { return nil, err } - if !rd.haveGenColumn { - return row, nil + var pkCols []int64 + if tbl.IsCommonHandle { + pkIdx := tables.FindPrimaryIndex(tbl) + for _, idxCol := range pkIdx.Columns { + pkCols = append(pkCols, tbl.Columns[idxCol.Offset].ID) + } } - + log.Warnf("3333333333333333333 ", rd.columns) for _, dCol := range rd.columns { colInfo := dCol.Col.ColumnInfo val, ok := row[colInfo.ID] + log.Warnf("44444444444444444444444444 ") if ok || dCol.GenExpr != nil { + log.Warnf("55555555555555555555 ", val.String()) rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) continue } - ok, err := rd.tryDecodeFromHandle(dCol, handle) + log.Warnf("2222222222222222222222222222222222 ", dCol.Col.String()) + ok, err := rd.tryDecodeFromHandle(dCol, handle, pkCols) if err != nil { return nil, err } if ok { + row[colInfo.ID] = rd.mutRow.ToRow().GetDatum(colInfo.Offset, &dCol.Col.FieldType) + log.Warnf("11111111111111111111111111111 ", dCol.Col.String()) continue } // Get the default value of the column in the generated column expression. diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index bd59619d2123d..f1c46599d712d 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -128,7 +128,7 @@ func (s *testDecoderSuite) TestRowDecoder(c *C) { c.Assert(err, IsNil) c.Assert(bs, NotNil) - r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) + r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil, tbl.Meta()) c.Assert(err, IsNil) // Last column is primary-key column, and the table primary-key is handle, then the primary-key value won't be // stored in raw data, but store in the raw key. @@ -146,7 +146,7 @@ func (s *testDecoderSuite) TestRowDecoder(c *C) { } } // test decode with no generated column. - r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) + r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil, tbl.Meta()) c.Assert(err, IsNil) for k, v := range r2 { v1, ok := r[k] From cc029f794e6cf153c963a607c95f5b70ed5f4a70 Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Thu, 18 Jun 2020 12:49:32 +0800 Subject: [PATCH 03/10] save Signed-off-by: wjhuang2016 --- ddl/index.go | 2 ++ util/rowDecoder/decoder.go | 19 ++++++++++++++----- 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 1be4900a46eb8..93af45bd2bccf 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -15,6 +15,7 @@ package ddl import ( "context" + "github.com/prometheus/common/log" "math" "strconv" "sync/atomic" @@ -881,6 +882,7 @@ func (w *addIndexWorker) getIndexRecord(handle kv.Handle, recordKey []byte, rawR } idxVal := make([]types.Datum, len(idxInfo.Columns)) for j, v := range idxInfo.Columns { + log.Warnf("xxxxxxxxxxxxxxxxxxxxxxxxx ", v.Name) col := cols[v.Offset] idxColumnVal, ok := w.rowMap[col.ID] if ok { diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 47014041b6d9f..094536c18066a 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -59,11 +59,11 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { haveGenCol = true } } - if !haveGenCol { - return &RowDecoder{ - colTypes: colFieldMap, - } - } + //if !haveGenCol { + // return &RowDecoder{ + // colTypes: colFieldMap, + // } + //} cols := tbl.Cols() tps := make([]*types.FieldType, len(cols)) @@ -190,6 +190,15 @@ func BuildFullDecodeColMap(indexedCols []*table.Column, t table.Table, genExprPr copy(pendingCols, indexedCols) decodeColMap := make(map[int64]Column, len(pendingCols)) + for _, col := range t.Cols() { + if mysql.HasPriKeyFlag(col.Flag) { + log.Warn("hhhhhhhhhhhhhhhhhhhhh") + decodeColMap[col.ID] = Column{ + Col: col, + } + } + } + for i := 0; i < len(pendingCols); i++ { col := pendingCols[i] if _, ok := decodeColMap[col.ID]; ok { From 91eb65211a1674742e5785001745aa563acf4046 Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Thu, 18 Jun 2020 14:40:17 +0800 Subject: [PATCH 04/10] done Signed-off-by: wjhuang2016 --- ddl/index.go | 2 -- executor/batch_checker.go | 9 ++------- executor/builder.go | 17 ++++------------- executor/point_get.go | 34 ++++++++++++++++++++++------------ planner/core/plan_to_pb.go | 13 ++----------- table/tables/tables.go | 27 +++++++++++++++++++++++++++ util/admin/admin.go | 10 +--------- util/rowDecoder/decoder.go | 38 +++++++------------------------------- 8 files changed, 65 insertions(+), 85 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 93af45bd2bccf..1be4900a46eb8 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -15,7 +15,6 @@ package ddl import ( "context" - "github.com/prometheus/common/log" "math" "strconv" "sync/atomic" @@ -882,7 +881,6 @@ func (w *addIndexWorker) getIndexRecord(handle kv.Handle, recordKey []byte, rawR } idxVal := make([]types.Datum, len(idxInfo.Columns)) for j, v := range idxInfo.Columns { - log.Warnf("xxxxxxxxxxxxxxxxxxxxxxxxx ", v.Name) col := cols[v.Offset] idxColumnVal, ok := w.rowMap[col.ID] if ok { diff --git a/executor/batch_checker.go b/executor/batch_checker.go index 59517bcb156b1..576dc44090a2f 100644 --- a/executor/batch_checker.go +++ b/executor/batch_checker.go @@ -86,13 +86,8 @@ func getKeysNeedCheck(ctx context.Context, sctx sessionctx.Context, t table.Tabl break } } - } - if t.Meta().IsCommonHandle { - pkIdx := tables.FindPrimaryIndex(t.Meta()) - cols := t.Cols() - for _, idxCol := range pkIdx.Columns { - handleCols = append(handleCols, cols[idxCol.Offset]) - } + } else { + handleCols = tables.TryGetCommonPkColumns(t) } var err error diff --git a/executor/builder.go b/executor/builder.go index d1f2d9cf2ecbd..f0c73e1f6e83c 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -1843,13 +1843,8 @@ func (b *executorBuilder) buildAnalyzeColumnsPushdown(task plannercore.AnalyzeCo CmsketchDepth: &depth, CmsketchWidth: &width, } - if task.TblInfo != nil && task.TblInfo.IsCommonHandle { - pkIdx := tables.FindPrimaryIndex(task.TblInfo) - var pkColIds []int64 - for _, idxCol := range pkIdx.Columns { - pkColIds = append(pkColIds, task.TblInfo.Columns[idxCol.Offset].ID) - } - e.analyzePB.ColReq.PrimaryColumnIds = pkColIds + if task.TblInfo != nil { + e.analyzePB.ColReq.PrimaryColumnIds = tables.TryGetCommonPkColumnIds(task.TblInfo) } b.err = plannercore.SetPBColumnsDefaultValue(b.ctx, e.analyzePB.ColReq.ColumnsInfo, cols) job := &statistics.AnalyzeJob{DBName: task.DBName, TableName: task.TableName, PartitionName: task.PartitionName, JobInfo: autoAnalyze + "analyze columns"} @@ -3015,12 +3010,8 @@ func newRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model } } if len(pkCols) == 0 { - if tbl.IsCommonHandle { - pkIdx := tables.FindPrimaryIndex(tbl) - for _, idxCol := range pkIdx.Columns { - pkCols = append(pkCols, tbl.Columns[idxCol.Offset].ID) - } - } else { + pkCols := tables.TryGetCommonPkColumnIds(tbl) + if len(pkCols) == 0 { pkCols = []int64{0} } } diff --git a/executor/point_get.go b/executor/point_get.go index 6bc59edfe4871..7d6ccc2239574 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -16,6 +16,7 @@ package executor import ( "context" + "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" @@ -354,6 +355,7 @@ func decodeRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv.Ha } func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv.Handle, rowVal []byte, chk *chunk.Chunk) error { + pkCols := tables.TryGetCommonPkColumnIds(tblInfo) colID2CutPos := make(map[int64]int, e.schema.Len()) for _, col := range e.schema.Columns { if _, ok := colID2CutPos[col.ID]; !ok { @@ -374,8 +376,11 @@ func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv chk.AppendNull(i) continue } - if tryDecodeFromHandle(tblInfo, i, col, handle, chk, decoder) { - continue + if ok, err := tryDecodeFromHandle(tblInfo, i, col, handle, chk, decoder, pkCols); err != nil { + return err + if ok { + continue + } } cutPos := colID2CutPos[col.ID] if len(cutVals[cutPos]) == 0 { @@ -395,23 +400,28 @@ func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv return nil } -func tryDecodeFromHandle(tblInfo *model.TableInfo, i int, col *expression.Column, handle kv.Handle, chk *chunk.Chunk, decoder *codec.Decoder) bool { +func tryDecodeFromHandle(tblInfo *model.TableInfo, i int, col *expression.Column, handle kv.Handle, chk *chunk.Chunk, decoder *codec.Decoder, pkCols []int64) (bool, error) { if tblInfo.PKIsHandle && mysql.HasPriKeyFlag(col.RetType.Flag) { chk.AppendInt64(i, handle.IntValue()) - return true + return true, nil } if col.ID == model.ExtraHandleID { chk.AppendInt64(i, handle.IntValue()) - return true - } - if tblInfo.IsCommonHandle && mysql.HasPriKeyFlag(col.RetType.Flag) { - _, err := decoder.DecodeOne(handle.EncodedCol(i), i, col.RetType) - if err != nil { - return false + return true, nil + } + // Try to decode common handle. + if mysql.HasPriKeyFlag(col.RetType.Flag) { + for i, hid := range pkCols { + if col.ID == hid { + _, err := decoder.DecodeOne(handle.EncodedCol(i), i, col.RetType) + if err != nil { + return false, errors.Trace(err) + } + return true, nil + } } - return true } - return false + return false, nil } func getColInfoByID(tbl *model.TableInfo, colID int64) *model.ColumnInfo { diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 3728f46279d47..5853f9bbf857c 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -104,13 +104,7 @@ func (p *PhysicalLimit) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { // ToPB implements PhysicalPlan ToPB interface. func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) { - var pkColIds []int64 - if p.Table.IsCommonHandle { - pkIdx := tables.FindPrimaryIndex(p.Table) - for _, idxCol := range pkIdx.Columns { - pkColIds = append(pkColIds, p.Table.Columns[idxCol.Offset].ID) - } - } + pkColIds := tables.TryGetCommonPkColumnIds(p.Table) tsExec := &tipb.TableScan{ TableId: p.Table.ID, Columns: util.ColumnsToProto(p.Columns, p.Table.PKIsHandle), @@ -161,10 +155,7 @@ func (p *PhysicalIndexScan) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) } var pkColIds []int64 if p.NeedCommonHandle { - pkIdx := tables.FindPrimaryIndex(p.Table) - for _, idxCol := range pkIdx.Columns { - pkColIds = append(pkColIds, p.Table.Columns[idxCol.Offset].ID) - } + pkColIds = tables.TryGetCommonPkColumnIds(p.Table) } idxExec := &tipb.IndexScan{ TableId: p.Table.ID, diff --git a/table/tables/tables.go b/table/tables/tables.go index 734db261e6e36..3217f09642d2e 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -485,6 +485,33 @@ func FindPrimaryIndex(tblInfo *model.TableInfo) *model.IndexInfo { return pkIdx } +// TryGetCommonPkColumnIds get the IDs of primary key column if the table has common handle. +func TryGetCommonPkColumnIds(tbl *model.TableInfo) []int64 { + var pkColIds []int64 + if !tbl.IsCommonHandle { + return nil + } + pkIdx := FindPrimaryIndex(tbl) + for _, idxCol := range pkIdx.Columns { + pkColIds = append(pkColIds, tbl.Columns[idxCol.Offset].ID) + } + return pkColIds +} + +// TryGetCommonPkColumns get the primary key columns if the table has common handle. +func TryGetCommonPkColumns(tbl table.Table) []*table.Column { + var pkCols []*table.Column + if !tbl.Meta().IsCommonHandle { + return nil + } + pkIdx := FindPrimaryIndex(tbl.Meta()) + cols := tbl.Cols() + for _, idxCol := range pkIdx.Columns { + pkCols = append(pkCols, cols[idxCol.Offset]) + } + return pkCols +} + // AddRecord implements table.Table AddRecord interface. func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...table.AddRecordOption) (recordID kv.Handle, err error) { var opt table.AddRecordOpt diff --git a/util/admin/admin.go b/util/admin/admin.go index b125b3d85f600..ff0947e6bcd9e 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -444,15 +444,7 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab } data := make([]types.Datum, 0, len(cols)) for _, col := range cols { - if col.IsPKHandleColumn(t.Meta()) { - if mysql.HasUnsignedFlag(col.Flag) { - data = append(data, types.NewUintDatum(uint64(handle.IntValue()))) - } else { - data = append(data, types.NewIntDatum(handle.IntValue())) - } - } else { - data = append(data, rowMap[col.ID]) - } + data = append(data, rowMap[col.ID]) } more, err := fn(handle.IntValue(), data, cols) if !more || err != nil { diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 094536c18066a..0eff10901e42b 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -14,12 +14,11 @@ package decoder import ( - "github.com/pingcap/parser/model" - log "github.com/sirupsen/logrus" "sort" "time" "github.com/pingcap/errors" + "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" @@ -59,11 +58,6 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { haveGenCol = true } } - //if !haveGenCol { - // return &RowDecoder{ - // colTypes: colFieldMap, - // } - //} cols := tbl.Cols() tps := make([]*types.FieldType, len(cols)) @@ -80,15 +74,17 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { } } -func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle, pkCols []int64) (bool, error) { +func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle, pkCols []int64, row map[int64]types.Datum) (bool, error) { if handle == nil { return false, nil } colInfo := dCol.Col.ColumnInfo if dCol.Col.IsPKHandleColumn(rd.tbl.Meta()) { if mysql.HasUnsignedFlag(colInfo.Flag) { + row[colInfo.ID] = types.NewUintDatum(uint64(handle.IntValue())) rd.mutRow.SetValue(colInfo.Offset, uint64(handle.IntValue())) } else { + row[colInfo.ID] = types.NewIntDatum(handle.IntValue()) rd.mutRow.SetValue(colInfo.Offset, handle.IntValue()) } return true, nil @@ -101,6 +97,7 @@ func (rd *RowDecoder) tryDecodeFromHandle(dCol Column, handle kv.Handle, pkCols if err != nil { return false, errors.Trace(err) } + row[colInfo.ID] = d rd.mutRow.SetValue(colInfo.Offset, d) return true, nil } @@ -120,31 +117,19 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. if err != nil { return nil, err } - var pkCols []int64 - if tbl.IsCommonHandle { - pkIdx := tables.FindPrimaryIndex(tbl) - for _, idxCol := range pkIdx.Columns { - pkCols = append(pkCols, tbl.Columns[idxCol.Offset].ID) - } - } - log.Warnf("3333333333333333333 ", rd.columns) + pkCols := tables.TryGetCommonPkColumnIds(tbl) for _, dCol := range rd.columns { colInfo := dCol.Col.ColumnInfo val, ok := row[colInfo.ID] - log.Warnf("44444444444444444444444444 ") if ok || dCol.GenExpr != nil { - log.Warnf("55555555555555555555 ", val.String()) rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) continue } - log.Warnf("2222222222222222222222222222222222 ", dCol.Col.String()) - ok, err := rd.tryDecodeFromHandle(dCol, handle, pkCols) + ok, err := rd.tryDecodeFromHandleAndSetRow(dCol, handle, pkCols, row) if err != nil { return nil, err } if ok { - row[colInfo.ID] = rd.mutRow.ToRow().GetDatum(colInfo.Offset, &dCol.Col.FieldType) - log.Warnf("11111111111111111111111111111 ", dCol.Col.String()) continue } // Get the default value of the column in the generated column expression. @@ -190,15 +175,6 @@ func BuildFullDecodeColMap(indexedCols []*table.Column, t table.Table, genExprPr copy(pendingCols, indexedCols) decodeColMap := make(map[int64]Column, len(pendingCols)) - for _, col := range t.Cols() { - if mysql.HasPriKeyFlag(col.Flag) { - log.Warn("hhhhhhhhhhhhhhhhhhhhh") - decodeColMap[col.ID] = Column{ - Col: col, - } - } - } - for i := 0; i < len(pendingCols); i++ { col := pendingCols[i] if _, ok := decodeColMap[col.ID]; ok { From 20ee9cd881e8b2908a55d3a723a79b7df1c2fa4e Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Thu, 18 Jun 2020 15:46:55 +0800 Subject: [PATCH 05/10] done Signed-off-by: wjhuang2016 --- ddl/db_test.go | 41 +++++++++++++++++++++++++++++++++++------ executor/builder.go | 2 +- executor/point_get.go | 9 +++++---- 3 files changed, 41 insertions(+), 11 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 4d784e60a5d38..a9ac006ce7c42 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -139,10 +139,8 @@ type testDBSuite6 struct{ *testDBSuite } type testDBSuite7 struct{ *testDBSuite } type testSerialDBSuite struct{ *testDBSuite } -func (s *testDBSuite4) TestAddIndexWithPK(c *C) { - s.tk = testkit.NewTestKit(c, s.store) - s.tk.MustExec("use " + s.schemaName) - +func testAddIndexWithPK(s *testSerialDBSuite, c *C) { + s.tk.MustExec("drop table if exists test_add_index_with_pk") s.tk.MustExec("create table test_add_index_with_pk(a int not null, b int not null default '0', primary key(a))") s.tk.MustExec("insert into test_add_index_with_pk values(1, 2)") s.tk.MustExec("alter table test_add_index_with_pk add index idx (a)") @@ -150,16 +148,35 @@ func (s *testDBSuite4) TestAddIndexWithPK(c *C) { s.tk.MustExec("insert into test_add_index_with_pk values(2, 2)") s.tk.MustExec("alter table test_add_index_with_pk add index idx1 (a, b)") s.tk.MustQuery("select * from test_add_index_with_pk").Check(testkit.Rows("1 2", "2 2")) + s.tk.MustExec("drop table if exists test_add_index_with_pk1") s.tk.MustExec("create table test_add_index_with_pk1(a int not null, b int not null default '0', c int, d int, primary key(c))") s.tk.MustExec("insert into test_add_index_with_pk1 values(1, 1, 1, 1)") s.tk.MustExec("alter table test_add_index_with_pk1 add index idx (c)") s.tk.MustExec("insert into test_add_index_with_pk1 values(2, 2, 2, 2)") s.tk.MustQuery("select * from test_add_index_with_pk1").Check(testkit.Rows("1 1 1 1", "2 2 2 2")) + s.tk.MustExec("drop table if exists test_add_index_with_pk2") s.tk.MustExec("create table test_add_index_with_pk2(a int not null, b int not null default '0', c int unsigned, d int, primary key(c))") s.tk.MustExec("insert into test_add_index_with_pk2 values(1, 1, 1, 1)") s.tk.MustExec("alter table test_add_index_with_pk2 add index idx (c)") s.tk.MustExec("insert into test_add_index_with_pk2 values(2, 2, 2, 2)") s.tk.MustQuery("select * from test_add_index_with_pk2").Check(testkit.Rows("1 1 1 1", "2 2 2 2")) + s.tk.MustExec("drop table if exists t") + s.tk.MustExec("create table t (a int, b int, c int, primary key(a, b));") + s.tk.MustExec("insert into t values (1, 2, 3);") + s.tk.MustExec("create index idx on t (a, b);") +} + +func (s *testSerialDBSuite) TestAddIndexWithPK(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.tk.MustExec("use " + s.schemaName) + save := config.GetGlobalConfig().AlterPrimaryKey + config.GetGlobalConfig().AlterPrimaryKey = false + + testAddIndexWithPK(s, c) + s.tk.MustExec("set @@tidb_enable_clustered_index = 1;") + testAddIndexWithPK(s, c) + + config.GetGlobalConfig().AlterPrimaryKey = save } func (s *testDBSuite1) TestRenameIndex(c *C) { @@ -3713,9 +3730,9 @@ func (s *testDBSuite4) TestIfExists(c *C) { s.tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1507|Error in list of partitions to p1")) } -func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) { - s.tk = testkit.NewTestKit(c, s.store) +func testAddIndexForGeneratedColumn(s *testSerialDBSuite, c *C) { s.tk.MustExec("use test_db") + s.tk.MustExec("drop table if exists t") s.tk.MustExec("create table t(y year NOT NULL DEFAULT '2155')") defer s.mustExec(c, "drop table t;") for i := 0; i < 50; i++ { @@ -3737,6 +3754,7 @@ func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) { //s.mustExec(c, "alter table t drop index idx_y") // Fix issue 9311. + s.tk.MustExec("drop table if exists gcai_table") s.tk.MustExec("create table gcai_table (id int primary key);") s.tk.MustExec("insert into gcai_table values(1);") s.tk.MustExec("ALTER TABLE gcai_table ADD COLUMN d date DEFAULT '9999-12-31';") @@ -3752,6 +3770,17 @@ func (s *testDBSuite5) TestAddIndexForGeneratedColumn(c *C) { s.tk.MustQuery("select id1 from gcai_table use index(idx1)").Check(testkit.Rows("6")) s.tk.MustExec("admin check table gcai_table") } +func (s *testSerialDBSuite) TestAddIndexForGeneratedColumn(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + save := config.GetGlobalConfig().AlterPrimaryKey + config.GetGlobalConfig().AlterPrimaryKey = false + + testAddIndexForGeneratedColumn(s, c) + s.tk.MustExec("set @@tidb_enable_clustered_index = 1;") + testAddIndexForGeneratedColumn(s, c) + + config.GetGlobalConfig().AlterPrimaryKey = save +} func (s *testDBSuite5) TestModifyGeneratedColumn(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/executor/builder.go b/executor/builder.go index f0c73e1f6e83c..b3f9e6d12d681 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -3010,7 +3010,7 @@ func newRowDecoder(ctx sessionctx.Context, schema *expression.Schema, tbl *model } } if len(pkCols) == 0 { - pkCols := tables.TryGetCommonPkColumnIds(tbl) + pkCols = tables.TryGetCommonPkColumnIds(tbl) if len(pkCols) == 0 { pkCols = []int64{0} } diff --git a/executor/point_get.go b/executor/point_get.go index 7d6ccc2239574..4c6ac84ccdf32 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -376,11 +376,12 @@ func decodeOldRowValToChunk(e *baseExecutor, tblInfo *model.TableInfo, handle kv chk.AppendNull(i) continue } - if ok, err := tryDecodeFromHandle(tblInfo, i, col, handle, chk, decoder, pkCols); err != nil { + ok, err := tryDecodeFromHandle(tblInfo, i, col, handle, chk, decoder, pkCols) + if err != nil { return err - if ok { - continue - } + } + if ok { + continue } cutPos := colID2CutPos[col.ID] if len(cutVals[cutPos]) == 0 { From 836599ca7208c8abfb8ecbc29a7322c5d772c1ee Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Thu, 18 Jun 2020 18:58:03 +0800 Subject: [PATCH 06/10] address comment Signed-off-by: wjhuang2016 --- ddl/db_test.go | 16 ++++++++-------- util/rowDecoder/decoder.go | 7 +++++-- 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index a9ac006ce7c42..db4062507b52f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -169,14 +169,14 @@ func testAddIndexWithPK(s *testSerialDBSuite, c *C) { func (s *testSerialDBSuite) TestAddIndexWithPK(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) - save := config.GetGlobalConfig().AlterPrimaryKey - config.GetGlobalConfig().AlterPrimaryKey = false + config.UpdateGlobal(func(conf *config.Config) { + conf.AlterPrimaryKey = false + }) + defer config.RestoreFunc()() testAddIndexWithPK(s, c) s.tk.MustExec("set @@tidb_enable_clustered_index = 1;") testAddIndexWithPK(s, c) - - config.GetGlobalConfig().AlterPrimaryKey = save } func (s *testDBSuite1) TestRenameIndex(c *C) { @@ -3772,14 +3772,14 @@ func testAddIndexForGeneratedColumn(s *testSerialDBSuite, c *C) { } func (s *testSerialDBSuite) TestAddIndexForGeneratedColumn(c *C) { s.tk = testkit.NewTestKit(c, s.store) - save := config.GetGlobalConfig().AlterPrimaryKey - config.GetGlobalConfig().AlterPrimaryKey = false + config.UpdateGlobal(func(conf *config.Config) { + conf.AlterPrimaryKey = false + }) + defer config.RestoreFunc()() testAddIndexForGeneratedColumn(s, c) s.tk.MustExec("set @@tidb_enable_clustered_index = 1;") testAddIndexForGeneratedColumn(s, c) - - config.GetGlobalConfig().AlterPrimaryKey = save } func (s *testDBSuite5) TestModifyGeneratedColumn(c *C) { diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index 0eff10901e42b..fdbfa66946b01 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -74,7 +74,7 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { } } -func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle, pkCols []int64, row map[int64]types.Datum) (bool, error) { +func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle, pkCols []int64, row map[int64]types.Datum, decodeLoc *time.Location) (bool, error) { if handle == nil { return false, nil } @@ -97,6 +97,9 @@ func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle if err != nil { return false, errors.Trace(err) } + if d, err = tablecodec.Unflatten(d, &dCol.Col.FieldType, decodeLoc); err != nil { + return false, err + } row[colInfo.ID] = d rd.mutRow.SetValue(colInfo.Offset, d) return true, nil @@ -125,7 +128,7 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) continue } - ok, err := rd.tryDecodeFromHandleAndSetRow(dCol, handle, pkCols, row) + ok, err := rd.tryDecodeFromHandleAndSetRow(dCol, handle, pkCols, row, decodeLoc) if err != nil { return nil, err } From 803ba4685d3c6e9e9e0b2eb9c79fc440fd09fcc6 Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Fri, 19 Jun 2020 16:42:30 +0800 Subject: [PATCH 07/10] address comment Signed-off-by: wjhuang2016 --- util/rowDecoder/decoder.go | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index fdbfa66946b01..d61209fe93ce3 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -46,6 +46,7 @@ type RowDecoder struct { colTypes map[int64]*types.FieldType haveGenColumn bool defaultVals []types.Datum + pkCols []int64 } // NewRowDecoder returns a new RowDecoder. @@ -71,10 +72,11 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { colTypes: colFieldMap, haveGenColumn: haveGenCol, defaultVals: make([]types.Datum, len(cols)), + pkCols: tables.TryGetCommonPkColumnIds(tbl), } } -func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle, pkCols []int64, row map[int64]types.Datum, decodeLoc *time.Location) (bool, error) { +func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle, row map[int64]types.Datum, decodeLoc *time.Location) (bool, error) { if handle == nil { return false, nil } @@ -91,7 +93,7 @@ func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle } // Try to decode common handle. if mysql.HasPriKeyFlag(dCol.Col.Flag) { - for i, hid := range pkCols { + for i, hid := range rd.pkCols { if dCol.Col.ID == hid { _, d, err := codec.DecodeOne(handle.EncodedCol(i)) if err != nil { @@ -120,7 +122,6 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. if err != nil { return nil, err } - pkCols := tables.TryGetCommonPkColumnIds(tbl) for _, dCol := range rd.columns { colInfo := dCol.Col.ColumnInfo val, ok := row[colInfo.ID] @@ -128,7 +129,7 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv. rd.mutRow.SetValue(colInfo.Offset, val.GetValue()) continue } - ok, err := rd.tryDecodeFromHandleAndSetRow(dCol, handle, pkCols, row, decodeLoc) + ok, err := rd.tryDecodeFromHandleAndSetRow(dCol, handle, row, decodeLoc) if err != nil { return nil, err } From 5c558dc8b4c0691f829c51fdd68bcbcb083f619f Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Fri, 19 Jun 2020 17:05:52 +0800 Subject: [PATCH 08/10] address comment Signed-off-by: wjhuang2016 --- ddl/index.go | 2 +- util/admin/admin.go | 2 +- util/rowDecoder/decoder.go | 2 +- util/rowDecoder/decoder_test.go | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 1be4900a46eb8..988a80e29291b 100755 --- a/ddl/index.go +++ b/ddl/index.go @@ -875,7 +875,7 @@ func (w *addIndexWorker) getIndexRecord(handle kv.Handle, recordKey []byte, rawR cols := t.Cols() idxInfo := w.index.Meta() sysZone := timeutil.SystemLocation() - _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, time.UTC, sysZone, w.rowMap, t.Meta()) + _, err := w.rowDecoder.DecodeAndEvalRowWithMap(w.sessCtx, handle, rawRecord, time.UTC, sysZone, w.rowMap) if err != nil { return nil, errors.Trace(errCantDecodeIndex.GenWithStackByArgs(err)) } diff --git a/util/admin/admin.go b/util/admin/admin.go index ff0947e6bcd9e..49ebda5524e18 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -438,7 +438,7 @@ func iterRecords(sessCtx sessionctx.Context, retriever kv.Retriever, t table.Tab return errors.Trace(err) } - rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), time.UTC, nil, t.Meta()) + rowMap, err := rowDecoder.DecodeAndEvalRowWithMap(sessCtx, handle, it.Value(), sessCtx.GetSessionVars().Location(), time.UTC, nil) if err != nil { return errors.Trace(err) } diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index d61209fe93ce3..a2e61a46e5cb5 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -112,7 +112,7 @@ func (rd *RowDecoder) tryDecodeFromHandleAndSetRow(dCol Column, handle kv.Handle } // DecodeAndEvalRowWithMap decodes a byte slice into datums and evaluates the generated column value. -func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum, tbl *model.TableInfo) (map[int64]types.Datum, error) { +func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, handle kv.Handle, b []byte, decodeLoc, sysLoc *time.Location, row map[int64]types.Datum) (map[int64]types.Datum, error) { var err error if rowcodec.IsNewFormat(b) { row, err = tablecodec.DecodeRowWithMapNew(b, rd.colTypes, decodeLoc, row) diff --git a/util/rowDecoder/decoder_test.go b/util/rowDecoder/decoder_test.go index f1c46599d712d..bd59619d2123d 100644 --- a/util/rowDecoder/decoder_test.go +++ b/util/rowDecoder/decoder_test.go @@ -128,7 +128,7 @@ func (s *testDecoderSuite) TestRowDecoder(c *C) { c.Assert(err, IsNil) c.Assert(bs, NotNil) - r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil, tbl.Meta()) + r, err := de.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) c.Assert(err, IsNil) // Last column is primary-key column, and the table primary-key is handle, then the primary-key value won't be // stored in raw data, but store in the raw key. @@ -146,7 +146,7 @@ func (s *testDecoderSuite) TestRowDecoder(c *C) { } } // test decode with no generated column. - r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil, tbl.Meta()) + r2, err := deWithNoGenCols.DecodeAndEvalRowWithMap(ctx, kv.IntHandle(i), bs, time.UTC, timeZoneIn8, nil) c.Assert(err, IsNil) for k, v := range r2 { v1, ok := r[k] From b1df5af0592ab9ed2cefd7e2500228a6bc4c2e7f Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Fri, 19 Jun 2020 17:11:01 +0800 Subject: [PATCH 09/10] fmt Signed-off-by: wjhuang2016 --- ddl/db_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index db4062507b52f..dab5be7ce8c54 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -139,7 +139,7 @@ type testDBSuite6 struct{ *testDBSuite } type testDBSuite7 struct{ *testDBSuite } type testSerialDBSuite struct{ *testDBSuite } -func testAddIndexWithPK(s *testSerialDBSuite, c *C) { +func testAddIndexWithPK(s *testSerialDBSuite, c *C) { s.tk.MustExec("drop table if exists test_add_index_with_pk") s.tk.MustExec("create table test_add_index_with_pk(a int not null, b int not null default '0', primary key(a))") s.tk.MustExec("insert into test_add_index_with_pk values(1, 2)") From 179b8f107fd5acfd5c756717d78612f2571ea7de Mon Sep 17 00:00:00 2001 From: wjhuang2016 Date: Fri, 19 Jun 2020 18:46:29 +0800 Subject: [PATCH 10/10] fix Signed-off-by: wjhuang2016 --- util/rowDecoder/decoder.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/util/rowDecoder/decoder.go b/util/rowDecoder/decoder.go index a2e61a46e5cb5..4e48f0f8e3d0b 100644 --- a/util/rowDecoder/decoder.go +++ b/util/rowDecoder/decoder.go @@ -18,7 +18,6 @@ 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/kv" @@ -72,7 +71,7 @@ func NewRowDecoder(tbl table.Table, decodeColMap map[int64]Column) *RowDecoder { colTypes: colFieldMap, haveGenColumn: haveGenCol, defaultVals: make([]types.Datum, len(cols)), - pkCols: tables.TryGetCommonPkColumnIds(tbl), + pkCols: tables.TryGetCommonPkColumnIds(tbl.Meta()), } }