Skip to content

Commit

Permalink
Merge pull request #2435 from tamird/sql-alter-table
Browse files Browse the repository at this point in the history
sql: ALTER TABLE ADD {,COLUMN,CONSTRAINT} support
  • Loading branch information
tamird committed Sep 11, 2015
2 parents 197d42c + bee8f54 commit d8e5095
Show file tree
Hide file tree
Showing 17 changed files with 5,857 additions and 5,368 deletions.
124 changes: 124 additions & 0 deletions sql/alter_table.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,124 @@
// Copyright 2015 The Cockroach Authors.
//
// 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License. See the AUTHORS file
// for names of contributors.
//
// Author: Tamir Duberstein (tamird@gmail.com)

package sql

import (
"fmt"

"github.com/cockroachdb/cockroach/sql/parser"
"github.com/cockroachdb/cockroach/sql/privilege"
"github.com/cockroachdb/cockroach/util"
)

// AlterTable creates a table.
// Privileges: CREATE on table.
// notes: postgres requires CREATE on the table.
// mysql requires ALTER, CREATE, INSERT on the table.
func (p *planner) AlterTable(n *parser.AlterTable) (planNode, error) {
if err := n.Table.NormalizeTableName(p.session.Database); err != nil {
return nil, err
}

dbDesc, err := p.getDatabaseDesc(n.Table.Database())
if err != nil {
return nil, err
}

// Check if table exists.
tbKey := tableKey{dbDesc.ID, n.Table.Table()}.Key()
gr, err := p.txn.Get(tbKey)
if err != nil {
return nil, err
}
if !gr.Exists() {
if n.IfExists {
// Noop.
return &valuesNode{}, nil
}
// Key does not exist, but we want it to: error out.
return nil, fmt.Errorf("table %q does not exist", n.Table.Table())
}

tableDesc, err := p.getTableDesc(n.Table)
if err != nil {
return nil, err
}

if err := p.checkPrivilege(tableDesc, privilege.CREATE); err != nil {
return nil, err
}

nextIndexID := tableDesc.NextIndexID

for _, cmd := range n.Cmds {
switch t := cmd.(type) {
case *parser.AlterTableAddColumn:
d := t.ColumnDef
col, idx, err := makeColumnDefDescs(d)
if err != nil {
return nil, err
}
tableDesc.AddColumn(*col)
if idx != nil {
if err := tableDesc.AddIndex(*idx, d.PrimaryKey); err != nil {
return nil, err
}
}
case *parser.AlterTableAddConstraint:
d := t.ConstraintDef
idx := IndexDescriptor{
Name: string(d.Name),
Unique: d.Unique,
ColumnNames: d.Columns,
StoreColumnNames: d.Storing,
}
if err := tableDesc.AddIndex(idx, d.PrimaryKey); err != nil {
return nil, err
}
default:
return nil, util.Errorf("unsupported alter cmd: %T", cmd)
}
}

if err := tableDesc.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 {
return nil, err
}

b.Put(MakeDescMetadataKey(tableDesc.GetID()), tableDesc)
// Mark transaction as operating on the system DB.
p.txn.SetSystemDBTrigger()

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

return &valuesNode{}, nil
}
81 changes: 81 additions & 0 deletions sql/backfill.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,81 @@
// Copyright 2015 The Cockroach Authors.
//
// 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,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
// implied. See the License for the specific language governing
// permissions and limitations under the License. See the AUTHORS file
// for names of contributors.
//
// Author: Tamir Duberstein (tamird@gmail.com)

package sql

import (
"github.com/cockroachdb/cockroach/client"
"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) {
var 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
}

// 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
}
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
}

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

return b, row.Err()
}
63 changes: 7 additions & 56 deletions sql/create.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,11 +20,9 @@ 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/cockroachdb/cockroach/util/log"
)

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

if err := tableDesc.AllocateIDs(); err != nil {
return nil, err
Expand All @@ -85,66 +85,17 @@ func (p *planner) CreateIndex(n *parser.CreateIndex) (planNode, error) {
// `indexDesc` changed on us when we called `tableDesc.AllocateIDs()`.
indexDesc = tableDesc.Indexes[len(tableDesc.Indexes)-1]

// 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: n.Table}},
})
b, err := p.makeBackfillBatch(n.Table, tableDesc, indexDesc)
if err != nil {
return nil, err
}

// 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 nil, err
}
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.
var b client.Batch
b.Put(MakeDescMetadataKey(tableDesc.GetID()), tableDesc)

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

secondaryIndexEntries, err := encodeSecondaryIndexes(
tableDesc.ID, []IndexDescriptor{indexDesc}, colIDtoRowIndex, rowVals)
if err != nil {
return nil, err
}

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

if err := row.Err(); err != nil {
return nil, err
}

// Mark transaction as operating on the system DB.
p.txn.SetSystemDBTrigger()
if err := runBatchWithErrorConversion(tableDesc, p.txn.Run, b); err != nil {
return nil, err

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

return &valuesNode{}, nil
Expand Down
48 changes: 22 additions & 26 deletions sql/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -43,34 +43,30 @@ func (e errUniquenessConstraintViolation) Error() string {
e.index.Name)
}

func runBatchWithErrorConversion(tableDesc *TableDescriptor, runBatch func(*client.Batch) error, b client.Batch) error {
if err := runBatch(&b); err != nil {
for _, result := range b.Results {
if _, ok := result.Err.(*proto.ConditionFailedError); ok {
for _, row := range result.Rows {
indexID, key, err := decodeIndexKeyPrefix(tableDesc, row.Key)
if err != nil {
return err
}
index, err := tableDesc.FindIndexByID(indexID)
if err != nil {
return err
}
valTypes, err := makeKeyVals(tableDesc, index.ColumnIDs)
if err != nil {
return err
}
vals := make([]parser.Datum, len(valTypes))
if _, err := decodeKeyVals(valTypes, vals, key); err != nil {
return err
}

return errUniquenessConstraintViolation{index: index, vals: vals}
func convertBatchError(tableDesc *TableDescriptor, b client.Batch, err error) error {
for _, result := range b.Results {
if _, ok := result.Err.(*proto.ConditionFailedError); ok {
for _, row := range result.Rows {
indexID, key, err := decodeIndexKeyPrefix(tableDesc, row.Key)
if err != nil {
return err
}
index, err := tableDesc.FindIndexByID(indexID)
if err != nil {
return err
}
valTypes, err := makeKeyVals(tableDesc, index.ColumnIDs)
if err != nil {
return err
}
vals := make([]parser.Datum, len(valTypes))
if _, err := decodeKeyVals(valTypes, vals, key); err != nil {
return err
}

return errUniquenessConstraintViolation{index: index, vals: vals}
}
}
return err
}

return nil
return err
}
4 changes: 2 additions & 2 deletions sql/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -147,8 +147,8 @@ func (p *planner) Insert(n *parser.Insert) (planNode, error) {
if err := rows.Err(); err != nil {
return nil, err
}
if err := runBatchWithErrorConversion(tableDesc, p.txn.Run, b); err != nil {
return nil, err
if err := p.txn.Run(&b); err != nil {
return nil, convertBatchError(tableDesc, b, err)
}
// TODO(tamird/pmattis): return the number of affected rows
return &valuesNode{}, nil
Expand Down
Loading

0 comments on commit d8e5095

Please sign in to comment.