Skip to content

Commit

Permalink
Merge pull request #2929 from tamird/sql-drop-column-index
Browse files Browse the repository at this point in the history
sql: add DROP {CONSTRAINT,COLUMN} support
  • Loading branch information
tamird committed Oct 28, 2015
2 parents f5adccd + 9e1d365 commit f95c001
Show file tree
Hide file tree
Showing 10 changed files with 293 additions and 131 deletions.
61 changes: 44 additions & 17 deletions sql/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@ package sql
import (
"fmt"

"github.com/cockroachdb/cockroach/client"
"github.com/cockroachdb/cockroach/sql/parser"
"github.com/cockroachdb/cockroach/sql/privilege"
"github.com/cockroachdb/cockroach/util"
"github.com/gogo/protobuf/proto"
)

// AlterTable creates a table.
Expand Down Expand Up @@ -63,8 +65,9 @@ func (p *planner) AlterTable(n *parser.AlterTable) (planNode, error) {
return nil, err
}

nextIndexID := tableDesc.NextIndexID
newTableDesc := proto.Clone(tableDesc).(*TableDescriptor)

b := client.Batch{}
for _, cmd := range n.Cmds {
switch t := cmd.(type) {
case *parser.AlterTableAddColumn:
Expand All @@ -73,9 +76,9 @@ func (p *planner) AlterTable(n *parser.AlterTable) (planNode, error) {
if err != nil {
return nil, err
}
tableDesc.AddColumn(*col)
newTableDesc.AddColumn(*col)
if idx != nil {
if err := tableDesc.AddIndex(*idx, d.PrimaryKey); err != nil {
if err := newTableDesc.AddIndex(*idx, d.PrimaryKey); err != nil {
return nil, err
}
}
Expand All @@ -89,39 +92,63 @@ func (p *planner) AlterTable(n *parser.AlterTable) (planNode, error) {
ColumnNames: d.Columns,
StoreColumnNames: d.Storing,
}
if err := tableDesc.AddIndex(idx, d.PrimaryKey); err != nil {
if err := newTableDesc.AddIndex(idx, d.PrimaryKey); err != nil {
return nil, err
}
default:
return nil, util.Errorf("unsupported constraint: %T", t.ConstraintDef)
}

case *parser.AlterTableDropColumn:
i, err := newTableDesc.FindColumnByName(t.Column)
if err != nil {
if t.IfExists {
// Noop.
continue
}
return nil, err
}

col := newTableDesc.Columns[i]
if newTableDesc.PrimaryIndex.containsColumnID(col.ID) {
return nil, fmt.Errorf("column %q is referenced by the primary key", col.Name)
}
for _, idx := range newTableDesc.Indexes {
if idx.containsColumnID(col.ID) {
return nil, fmt.Errorf("column %q is referenced by existing index %q", col.Name, idx.Name)
}
}

newTableDesc.Columns = append(newTableDesc.Columns[:i], newTableDesc.Columns[i+1:]...)

case *parser.AlterTableDropConstraint:
i, err := newTableDesc.FindIndexByName(t.Constraint)
if err != nil {
if t.IfExists {
// Noop.
continue
}
return nil, err
}
newTableDesc.Indexes = append(newTableDesc.Indexes[:i], newTableDesc.Indexes[i+1:]...)

default:
return nil, util.Errorf("unsupported alter cmd: %T", cmd)
}
}

if err := tableDesc.AllocateIDs(); err != nil {
if err := newTableDesc.AllocateIDs(); err != nil {
return nil, err
}

// These changed on us when we called `tableDesc.AllocateIDs()`.
var newIndexes []IndexDescriptor
for _, index := range append(tableDesc.Indexes, tableDesc.PrimaryIndex) {
if index.ID >= nextIndexID {
newIndexes = append(newIndexes, index)
}
}

b, err := p.makeBackfillBatch(n.Table, tableDesc, newIndexes...)
if err != nil {
if err := p.backfillBatch(&b, n.Table, tableDesc, newTableDesc); err != nil {
return nil, err
}

b.Put(MakeDescMetadataKey(tableDesc.GetID()), tableDesc)
b.Put(MakeDescMetadataKey(newTableDesc.GetID()), newTableDesc)

if err := p.txn.Run(&b); err != nil {
return nil, convertBatchError(tableDesc, b, err)
return nil, convertBatchError(newTableDesc, b, err)
}

return &valuesNode{}, nil
Expand Down
194 changes: 149 additions & 45 deletions sql/backfill.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,65 +18,169 @@
package sql

import (
"sort"

"github.com/cockroachdb/cockroach/client"
"github.com/cockroachdb/cockroach/roachpb"
"github.com/cockroachdb/cockroach/sql/parser"
"github.com/cockroachdb/cockroach/util/log"
)

func (p *planner) makeBackfillBatch(tableName *parser.QualifiedName, tableDesc *TableDescriptor, indexDescs ...IndexDescriptor) (client.Batch, error) {
b := client.Batch{}
// Get all the rows affected.
// TODO(vivek): Avoid going through Select.
// TODO(tamird): Support partial indexes?
row, err := p.Select(&parser.Select{
Exprs: parser.SelectExprs{parser.StarSelectExpr()},
From: parser.TableExprs{&parser.AliasedTableExpr{Expr: tableName}},
})
if err != nil {
return b, err
func makeColIDtoRowIndex(row planNode, desc *TableDescriptor) (map[ColumnID]int, error) {
columns := row.Columns()
colIDtoRowIndex := make(map[ColumnID]int, len(columns))
for i, name := range columns {
j, err := desc.FindColumnByName(name)
if err != nil {
return nil, err
}
colIDtoRowIndex[desc.Columns[j].ID] = i
}
return colIDtoRowIndex, nil
}

// Construct a map from column ID to the index the value appears at within a
// row.
colIDtoRowIndex := map[ColumnID]int{}
for i, name := range row.Columns() {
c, err := tableDesc.FindColumnByName(name)
if err != nil {
return b, err
var _ sort.Interface = columnsByID{}
var _ sort.Interface = indexesByID{}

type columnsByID []ColumnDescriptor

func (cds columnsByID) Len() int {
return len(cds)
}
func (cds columnsByID) Less(i, j int) bool {
return cds[i].ID < cds[j].ID
}
func (cds columnsByID) Swap(i, j int) {
cds[i], cds[j] = cds[j], cds[i]
}

type indexesByID []IndexDescriptor

func (ids indexesByID) Len() int {
return len(ids)
}
func (ids indexesByID) Less(i, j int) bool {
return ids[i].ID < ids[j].ID
}
func (ids indexesByID) Swap(i, j int) {
ids[i], ids[j] = ids[j], ids[i]
}

func (p *planner) backfillBatch(b *client.Batch, tableName *parser.QualifiedName, oldTableDesc, newTableDesc *TableDescriptor) error {
table := &parser.AliasedTableExpr{Expr: tableName}

var droppedColumnDescs []ColumnDescriptor
sort.Sort(columnsByID(oldTableDesc.Columns))
sort.Sort(columnsByID(newTableDesc.Columns))
for i, j := 0, 0; i < len(oldTableDesc.Columns); i++ {
if j == len(newTableDesc.Columns) || oldTableDesc.Columns[i].ID != newTableDesc.Columns[j].ID {
droppedColumnDescs = append(droppedColumnDescs, oldTableDesc.Columns[i])
} else {
j++
}
colIDtoRowIndex[c.ID] = i
}

// TODO(tamird): This will fall down in production use. We need to do
// something better (see #2036). In particular, this implementation
// has the following problems:
// - Very large tables will generate an enormous batch here. This
// isn't really a problem in itself except that it will exacerbate
// the other issue:
// - Any non-quiescent table that this runs against will end up with
// an inconsistent index. This is because as inserts/updates continue
// to roll in behind this operation's read front, the written index
// will become incomplete/stale before it's written.

for row.Next() {
rowVals := row.Values()

for _, indexDesc := range indexDescs {
secondaryIndexEntries, err := encodeSecondaryIndexes(
tableDesc.ID, []IndexDescriptor{indexDesc}, colIDtoRowIndex, rowVals)
if err != nil {
return b, err
}
if len(droppedColumnDescs) > 0 {
var updateExprs parser.UpdateExprs
for _, droppedColumnDesc := range droppedColumnDescs {
updateExprs = append(updateExprs, &parser.UpdateExpr{
Names: parser.QualifiedNames{&parser.QualifiedName{Base: parser.Name(droppedColumnDesc.Name)}},
Expr: parser.DNull,
})
}

// Run `UPDATE <table> SET col1 = NULL, col2 = NULL, ...` to clear
// the data stored in the columns being dropped.
if _, err := p.Update(&parser.Update{
Table: table,
Exprs: updateExprs,
}); err != nil {
return err
}
}

for _, secondaryIndexEntry := range secondaryIndexEntries {
if log.V(2) {
log.Infof("CPut %s -> %v", prettyKey(secondaryIndexEntry.key, 0),
secondaryIndexEntry.value)
var droppedIndexDescs []IndexDescriptor
sort.Sort(indexesByID(oldTableDesc.Indexes))
sort.Sort(indexesByID(newTableDesc.Indexes))
for i, j := 0, 0; i < len(oldTableDesc.Indexes); i++ {
if j == len(newTableDesc.Indexes) || oldTableDesc.Indexes[i].ID != newTableDesc.Indexes[j].ID {
droppedIndexDescs = append(droppedIndexDescs, oldTableDesc.Indexes[i])
} else {
j++
}
}

for _, indexDescriptor := range droppedIndexDescs {
indexPrefix := MakeIndexKeyPrefix(newTableDesc.ID, indexDescriptor.ID)

// Delete the index.
indexStartKey := roachpb.Key(indexPrefix)
indexEndKey := indexStartKey.PrefixEnd()
if log.V(2) {
log.Infof("DelRange %s - %s", prettyKey(indexStartKey, 0), prettyKey(indexEndKey, 0))
}
b.DelRange(indexStartKey, indexEndKey)
}

var newIndexDescs []IndexDescriptor
for _, index := range append(newTableDesc.Indexes, newTableDesc.PrimaryIndex) {
if index.ID >= oldTableDesc.NextIndexID {
newIndexDescs = append(newIndexDescs, index)
}
}

if len(newIndexDescs) > 0 {
// Get all the rows affected.
// TODO(vivek): Avoid going through Select.
// TODO(tamird): Support partial indexes?
rows, err := p.Select(&parser.Select{
Exprs: parser.SelectExprs{parser.StarSelectExpr()},
From: parser.TableExprs{table},
})
if err != nil {
return err
}

// Construct a map from column ID to the index the value appears at within a
// row.
colIDtoRowIndex, err := makeColIDtoRowIndex(rows, oldTableDesc)
if err != nil {
return err
}

// TODO(tamird): This will fall down in production use. We need to do
// something better (see #2036). In particular, this implementation
// has the following problems:
// - Very large tables will generate an enormous batch here. This
// isn't really a problem in itself except that it will exacerbate
// the other issue:
// - Any non-quiescent table that this runs against will end up with
// an inconsistent index. This is because as inserts/updates continue
// to roll in behind this operation's read front, the written index
// will become incomplete/stale before it's written.

for rows.Next() {
rowVals := rows.Values()

for _, newIndexDesc := range newIndexDescs {
secondaryIndexEntries, err := encodeSecondaryIndexes(
oldTableDesc.ID, []IndexDescriptor{newIndexDesc}, colIDtoRowIndex, rowVals)
if err != nil {
return err
}

for _, secondaryIndexEntry := range secondaryIndexEntries {
if log.V(2) {
log.Infof("CPut %s -> %v", prettyKey(secondaryIndexEntry.key, 0),
secondaryIndexEntry.value)
}
b.CPut(secondaryIndexEntry.key, secondaryIndexEntry.value, nil)
}
b.CPut(secondaryIndexEntry.key, secondaryIndexEntry.value, nil)
}
}

return rows.Err()
}

return b, row.Err()
return nil
}
20 changes: 11 additions & 9 deletions sql/create.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,9 +20,11 @@ package sql
import (
"fmt"

"github.com/cockroachdb/cockroach/client"
"github.com/cockroachdb/cockroach/security"
"github.com/cockroachdb/cockroach/sql/parser"
"github.com/cockroachdb/cockroach/sql/privilege"
"github.com/gogo/protobuf/proto"
)

// CreateDatabase creates a database.
Expand Down Expand Up @@ -74,26 +76,26 @@ func (p *planner) CreateIndex(n *parser.CreateIndex) (planNode, error) {
ColumnNames: n.Columns,
StoreColumnNames: n.Storing,
}
if err := tableDesc.AddIndex(indexDesc, false); err != nil {

newTableDesc := proto.Clone(tableDesc).(*TableDescriptor)

if err := newTableDesc.AddIndex(indexDesc, false); err != nil {
return nil, err
}

if err := tableDesc.AllocateIDs(); err != nil {
if err := newTableDesc.AllocateIDs(); err != nil {
return nil, err
}

// `indexDesc` changed on us when we called `tableDesc.AllocateIDs()`.
indexDesc = tableDesc.Indexes[len(tableDesc.Indexes)-1]

b, err := p.makeBackfillBatch(n.Table, tableDesc, indexDesc)
if err != nil {
b := client.Batch{}
if err := p.backfillBatch(&b, n.Table, tableDesc, newTableDesc); err != nil {
return nil, err
}

b.Put(MakeDescMetadataKey(tableDesc.GetID()), tableDesc)
b.Put(MakeDescMetadataKey(newTableDesc.GetID()), newTableDesc)

if err := p.txn.Run(&b); err != nil {
return nil, convertBatchError(tableDesc, b, err)
return nil, convertBatchError(newTableDesc, b, err)
}

return &valuesNode{}, nil
Expand Down
Loading

0 comments on commit f95c001

Please sign in to comment.