Skip to content

Commit

Permalink
ddl: fix invalid index on multi-layer virtual columns (pingcap#11260)
Browse files Browse the repository at this point in the history
  • Loading branch information
tangenta committed Jul 31, 2019
1 parent b4d9151 commit 6ec8dea
Show file tree
Hide file tree
Showing 5 changed files with 216 additions and 42 deletions.
64 changes: 64 additions & 0 deletions ddl/db_integration_test.go
Expand Up @@ -454,6 +454,70 @@ func (s *testIntegrationSuite) TestModifyingColumnOption(c *C) {
assertErrCode("alter table t2 modify column c int references t1(a)", errMsg)
}

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

tk.MustExec("create database if not exists test")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (c)")
tk.MustQuery("select * from t where c > 1").Check(testkit.Rows("1 2 3"))
res := tk.MustQuery("select * from t use index(idx) where c > 1")
tk.MustQuery("select * from t ignore index(idx) where c > 1").Check(res.Rows())
tk.MustExec("admin check table t")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int as (a + 1), c int as (b + 1), d int as (c + 1))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4"))
res = tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a bigint, b decimal as (a+1), c varchar(20) as (b*2), d float as (a*23+b-1+length(c)))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 4 25"))
res = tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a varchar(10), b float as (length(a)+123), c varchar(20) as (right(a, 2)), d float as (b+b-7+1-3+3*ASCII(c)))")
tk.MustExec("insert into t (a) values ('adorable')")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("adorable 131 le 577")) // 131+131-7+1-3+3*108
res = tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a bigint, b decimal as (a), c int(10) as (a+b), d float as (a+b+c), e decimal as (a+b+c+d))")
tk.MustExec("insert into t (a) values (1)")
tk.MustExec("create index idx on t (d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 1 2 4 8"))
res = tk.MustQuery("select * from t use index(idx) where d > 2")
tk.MustQuery("select * from t ignore index(idx) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")

tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a bigint, b bigint as (a+1) virtual, c bigint as (b+1) virtual)")
tk.MustExec("alter table t add index idx_b(b)")
tk.MustExec("alter table t add index idx_c(c)")
tk.MustExec("insert into t(a) values(1)")
tk.MustExec("alter table t add column(d bigint as (c+1) virtual)")
tk.MustExec("alter table t add index idx_d(d)")
tk.MustQuery("select * from t where d > 2").Check(testkit.Rows("1 2 3 4"))

res = tk.MustQuery("select * from t use index(idx_d) where d > 2")
tk.MustQuery("select * from t ignore index(idx_d) where d > 2").Check(res.Rows())
tk.MustExec("admin check table t")
}

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

Expand Down
38 changes: 17 additions & 21 deletions ddl/index.go
Expand Up @@ -884,27 +884,23 @@ func (w *addIndexWorker) run(d *ddlCtx) {

func makeupDecodeColMap(sessCtx sessionctx.Context, t table.Table, indexInfo *model.IndexInfo) (map[int64]decoder.Column, error) {
cols := t.Cols()
decodeColMap := make(map[int64]decoder.Column, len(indexInfo.Columns))
for _, v := range indexInfo.Columns {
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 {
decodeColMap[c.ID] = decoder.Column{
Info: c.ToInfo(),
}
}
}
e, err := expression.ParseSimpleExprCastWithTableInfo(sessCtx, col.GeneratedExprString, t.Meta(), &col.FieldType)
if err != nil {
return nil, errors.Trace(err)
}
tpExpr.GenExpr = e
}
decodeColMap[col.ID] = tpExpr
indexedCols := make([]*table.Column, len(indexInfo.Columns))
for i, v := range indexInfo.Columns {
indexedCols[i] = cols[v.Offset]
}

var containsVirtualCol bool
decodeColMap, err := decoder.BuildFullDecodeColMap(indexedCols, t, func(genCol *table.Column) (expression.Expression, error) {
containsVirtualCol = true
return expression.ParseSimpleExprCastWithTableInfo(sessCtx, genCol.GeneratedExprString, t.Meta(), &genCol.FieldType)
})
if err != nil {
return nil, err
}

if containsVirtualCol {
decoder.SubstituteGenColsInDecodeColMap(decodeColMap)
decoder.RemoveUnusedVirtualCols(decodeColMap, indexedCols)
}
return decodeColMap, nil
}
Expand Down
11 changes: 11 additions & 0 deletions planner/core/logical_plan_builder.go
Expand Up @@ -2058,6 +2058,17 @@ func (b *planBuilder) projectVirtualColumns(ds *DataSource, columns []*table.Col
}
proj.Exprs = append(proj.Exprs, expr)
}

// Re-iterate expressions to handle those virtual generated columns that refers to the other generated columns, for
// example, given:
// column a, column b as (a * 2), column c as (b + 1)
// we'll get:
// column a, column b as (a * 2), column c as ((a * 2) + 1)
// A generated column definition can refer to only generated columns occurring earlier in the table definition, so
// it's safe to iterate in index-ascending order.
for i, expr := range proj.Exprs {
proj.Exprs[i] = expression.ColumnSubstitute(expr, ds.Schema(), proj.Exprs)
}
proj.SetSchema(ds.Schema().Clone())
return proj, nil
}
Expand Down
33 changes: 12 additions & 21 deletions util/admin/admin.go
Expand Up @@ -595,27 +595,18 @@ func CompareTableRecord(sessCtx sessionctx.Context, txn kv.Transaction, t table.
}

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)
var containsVirtualCol bool
decodeColsMap, ignored := decoder.BuildFullDecodeColMap(decodeCol, t, func(genCol *table.Column) (expression.Expression, error) {
containsVirtualCol = true
return genExpr[model.TableColumnID{TableID: t.Meta().ID, ColumnID: genCol.ID}], nil
})
_ = ignored

if containsVirtualCol {
decoder.SubstituteGenColsInDecodeColMap(decodeColsMap)
decoder.RemoveUnusedVirtualCols(decodeColsMap, decodeCol)
}
return decoder.NewRowDecoder(t.Cols(), decodeColsMap)
}

// genExprs use to calculate generated column value.
Expand Down
112 changes: 112 additions & 0 deletions util/rowDecoder/decoder.go
Expand Up @@ -14,6 +14,7 @@
package decoder

import (
"sort"
"time"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -110,3 +111,114 @@ func (rd *RowDecoder) DecodeAndEvalRowWithMap(ctx sessionctx.Context, b []byte,
}
return row, nil
}

// BuildFullDecodeColMap build a map that contains [columnID -> struct{*table.Column, expression.Expression}] from
// indexed columns and all of its depending columns. `genExprProducer` is used to produce a generated expression based on a table.Column.
func BuildFullDecodeColMap(indexedCols []*table.Column, t table.Table, genExprProducer func(*table.Column) (expression.Expression, error)) (map[int64]Column, error) {
pendingCols := make([]*table.Column, len(indexedCols))
copy(pendingCols, indexedCols)
decodeColMap := make(map[int64]Column, len(pendingCols))

for i := 0; i < len(pendingCols); i++ {
col := pendingCols[i]
if _, ok := decodeColMap[col.ID]; ok {
continue // already discovered
}

if col.IsGenerated() && !col.GeneratedStored {
// Find depended columns and put them into pendingCols. For example, idx(c) with column definition `c int as (a + b)`,
// depended columns of `c` is `a` and `b`, and both of them will be put into the pendingCols, waiting for next traversal.
for _, c := range t.Cols() {
if _, ok := col.Dependences[c.Name.L]; ok {
pendingCols = append(pendingCols, c)
}
}

e, err := genExprProducer(col)
if err != nil {
return nil, errors.Trace(err)
}
decodeColMap[col.ID] = Column{
Info: col.ColumnInfo,
GenExpr: e,
}
} else {
decodeColMap[col.ID] = Column{
Info: col.ColumnInfo,
}
}
}
return decodeColMap, nil
}

// SubstituteGenColsInDecodeColMap substitutes generated columns in every expression
// with non-generated one by looking up decodeColMap.
func SubstituteGenColsInDecodeColMap(decodeColMap map[int64]Column) {
// Sort columns by table.Column.Offset in ascending order.
type Pair struct {
colID int64
colOffset int
}
var orderedCols []Pair
for colID, col := range decodeColMap {
orderedCols = append(orderedCols, Pair{colID, col.Info.Offset})
}
sort.Slice(orderedCols, func(i, j int) bool { return orderedCols[i].colOffset < orderedCols[j].colOffset })

// Iterate over decodeColMap, the substitution only happens once for each virtual column because
// columns with smaller offset can not refer to those with larger ones. https://dev.mysql.com/doc/refman/5.7/en/create-table-generated-columns.html.
for _, pair := range orderedCols {
colID := pair.colID
decCol := decodeColMap[colID]
if decCol.GenExpr != nil {
decodeColMap[colID] = Column{
Info: decCol.Info,
GenExpr: substituteGeneratedColumn(decCol.GenExpr, decodeColMap),
}
} else {
decodeColMap[colID] = Column{
Info: decCol.Info,
}
}
}
}

// substituteGeneratedColumn substitutes generated columns in an expression with non-generated one by looking up decodeColMap.
func substituteGeneratedColumn(expr expression.Expression, decodeColMap map[int64]Column) expression.Expression {
switch v := expr.(type) {
case *expression.Column:
if c, ok := decodeColMap[v.ID]; c.GenExpr != nil && ok {
return c.GenExpr
}
return v
case *expression.ScalarFunction:
newArgs := make([]expression.Expression, 0, len(v.GetArgs()))
for _, arg := range v.GetArgs() {
newArgs = append(newArgs, substituteGeneratedColumn(arg, decodeColMap))
}
return expression.NewFunctionInternal(v.GetCtx(), v.FuncName.L, v.RetType, newArgs...)
}
return expr
}

// RemoveUnusedVirtualCols removes all virtual columns in decodeColMap that cannot found in indexedCols.
func RemoveUnusedVirtualCols(decodeColMap map[int64]Column, indexedCols []*table.Column) {
for colID, decCol := range decodeColMap {
col := decCol.Info
if !col.IsGenerated() || col.GeneratedStored {
continue
}

found := false
for _, v := range indexedCols {
if v.Offset == col.Offset {
found = true
break
}
}

if !found {
delete(decodeColMap, colID)
}
}
}

0 comments on commit 6ec8dea

Please sign in to comment.