Skip to content

Commit

Permalink
sql: add support for CREATE OR REPLACE VIEW
Browse files Browse the repository at this point in the history
Fixes #24897.

This PR adds support for the CREATE OR REPLACE VIEW
command by allowing the create view statement to optionally
overwrite an existing descriptor, rather than always write
a new descriptor k/v entry.

Release note (sql change): This PR adds support for the
CREATE OR REPLACE VIEW command.
  • Loading branch information
rohany committed Apr 5, 2020
1 parent 5d33d44 commit 02c77fd
Show file tree
Hide file tree
Showing 15 changed files with 265 additions and 42 deletions.
2 changes: 2 additions & 0 deletions docs/generated/sql/bnf/create_view_stmt.bnf
@@ -1,5 +1,7 @@
create_view_stmt ::=
'CREATE' opt_temp 'VIEW' view_name '(' name_list ')' 'AS' select_stmt
| 'CREATE' opt_temp 'VIEW' view_name 'AS' select_stmt
| 'CREATE' 'OR' 'REPLACE' opt_temp 'VIEW' view_name '(' name_list ')' 'AS' select_stmt
| 'CREATE' 'OR' 'REPLACE' opt_temp 'VIEW' view_name 'AS' select_stmt
| 'CREATE' opt_temp 'VIEW' 'IF' 'NOT' 'EXISTS' view_name '(' name_list ')' 'AS' select_stmt
| 'CREATE' opt_temp 'VIEW' 'IF' 'NOT' 'EXISTS' view_name 'AS' select_stmt
1 change: 1 addition & 0 deletions docs/generated/sql/bnf/stmt_block.bnf
Expand Up @@ -1042,6 +1042,7 @@ create_table_as_stmt ::=

create_view_stmt ::=
'CREATE' opt_temp 'VIEW' view_name opt_column_list 'AS' select_stmt
| 'CREATE' 'OR' 'REPLACE' opt_temp 'VIEW' view_name opt_column_list 'AS' select_stmt
| 'CREATE' opt_temp 'VIEW' 'IF' 'NOT' 'EXISTS' view_name opt_column_list 'AS' select_stmt

create_sequence_stmt ::=
Expand Down
6 changes: 4 additions & 2 deletions pkg/sql/create_table.go
Expand Up @@ -118,7 +118,8 @@ var storageParamExpectedTypes = map[string]storageParamType{
func (n *createTableNode) ReadingOwnWrites() {}

// getTableCreateParams returns the table key needed for the new table,
// as well as the schema id.
// as well as the schema id. It returns valid data in the case that
// the desired object exists.
func getTableCreateParams(
params runParams, dbID sqlbase.ID, isTemporary bool, tableName string,
) (sqlbase.DescriptorKey, sqlbase.ID, error) {
Expand Down Expand Up @@ -161,7 +162,8 @@ func getTableCreateParams(

exists, _, err := sqlbase.LookupObjectID(params.ctx, params.p.txn, dbID, schemaID, tableName)
if err == nil && exists {
return nil, 0, sqlbase.NewRelationAlreadyExistsError(tableName)
// Still return data in this case.
return tKey, schemaID, sqlbase.NewRelationAlreadyExistsError(tableName)
} else if err != nil {
return nil, 0, err
}
Expand Down
207 changes: 169 additions & 38 deletions pkg/sql/create_view.go
Expand Up @@ -15,6 +15,7 @@ import (
"fmt"

"github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sqlbase"
Expand All @@ -30,6 +31,7 @@ type createViewNode struct {
// qualified.
viewQuery string
ifNotExists bool
replace bool
temporary bool
dbDesc *sqlbase.DatabaseDescriptor
columns sqlbase.ResultColumns
Expand Down Expand Up @@ -70,12 +72,34 @@ func (n *createViewNode) startExec(params runParams) error {
backRefMutables[id] = backRefMutable
}

var replacingDesc *sqlbase.MutableTableDescriptor

tKey, schemaID, err := getTableCreateParams(params, n.dbDesc.ID, isTemporary, viewName)
if err != nil {
if sqlbase.IsRelationAlreadyExistsError(err) && n.ifNotExists {
return nil
if sqlbase.IsRelationAlreadyExistsError(err) {
if n.ifNotExists {
return nil
} else if n.replace {
// If we are replacing an existing view see if what we are
// replacing is actually a view.
id, err := getDescriptorID(params.ctx, params.p.txn, tKey)
if err != nil {
return err
}
desc, err := params.p.Tables().getMutableTableVersionByID(params.ctx, id, params.p.txn)
if err != nil {
return err
}
if !desc.IsView() {
return pgerror.Newf(pgcode.WrongObjectType, `"%s" is not a view`, viewName)
}
replacingDesc = desc
} else {
return err
}
} else {
return err
}
return err
}

schemaName := tree.PublicSchemaName
Expand All @@ -84,42 +108,138 @@ func (n *createViewNode) startExec(params runParams) error {
schemaName = tree.Name(params.p.TemporarySchemaName())
}

id, err := GenerateUniqueDescID(params.ctx, params.extendedEvalCtx.ExecCfg.DB)
if err != nil {
return err
var id sqlbase.ID
if replacingDesc != nil {
id = replacingDesc.ID
} else {
var err error
id, err = GenerateUniqueDescID(params.ctx, params.extendedEvalCtx.ExecCfg.DB)
if err != nil {
return err
}
}

// Inherit permissions from the database descriptor.
privs := n.dbDesc.GetPrivileges()

desc, err := makeViewTableDesc(
viewName,
n.viewQuery,
n.dbDesc.ID,
schemaID,
id,
n.columns,
params.creationTimeForNewTableDescriptor(),
privs,
&params.p.semaCtx,
isTemporary,
)
if err != nil {
return err
}
var newDesc *sqlbase.MutableTableDescriptor

// Collect all the tables/views this view depends on.
for backrefID := range n.planDeps {
desc.DependsOn = append(desc.DependsOn, backrefID)
}
if replacingDesc != nil {
// Set the query to the new query.
replacingDesc.ViewQuery = n.viewQuery
// Reset the columns to add the new result columns onto.
replacingDesc.Columns = make([]sqlbase.ColumnDescriptor, 0, len(n.columns))
replacingDesc.NextColumnID = 0
if err := addResultColumns(&params.p.semaCtx, replacingDesc, n.columns); err != nil {
return err
}

// TODO (lucy): I think this needs a NodeFormatter implementation. For now,
// do some basic string formatting (not accurate in the general case).
if err = params.p.createDescriptorWithID(
params.ctx, tKey.Key(), id, &desc, params.EvalContext().Settings,
fmt.Sprintf("CREATE VIEW %q AS %q", n.viewName, n.viewQuery),
); err != nil {
return err
oldDesc := &replacingDesc.ClusterVersion

// Compare replacingDesc against its ClusterVersion to verify if
// its new set of columns is correct. The new view must have at
// least the same prefix of columns as the old view.
// We attempt to match the postgres error message in each of the
// error cases below.
if len(replacingDesc.Columns) < len(oldDesc.Columns) {
// Match the postgres error.
return pgerror.Newf(pgcode.InvalidTableDefinition, "cannot drop columns from view")
}
for i := range oldDesc.Columns {
oldCol, newCol := &oldDesc.Columns[i], &replacingDesc.Columns[i]
if oldCol.Name != newCol.Name {
return pgerror.Newf(
pgcode.InvalidTableDefinition,
`cannot change name of view column "%s" to "%s"`,
oldCol.Name,
newCol.Name,
)
}
if !newCol.Type.Equal(oldCol.Type) {
return pgerror.Newf(
pgcode.InvalidTableDefinition,
`cannot change type of view column "%s" from %s to %s`,
oldCol.Name,
oldCol.Type.String(),
newCol.Type.String(),
)
}
}

// Remove this view from all tables that depend on it.
for _, id := range replacingDesc.DependsOn {
desc, ok := backRefMutables[id]
if !ok {
var err error
desc, err = params.p.Tables().getMutableTableVersionByID(params.ctx, id, params.p.txn)
if err != nil {
return err
}
backRefMutables[id] = desc
}

// Remove the back reference.
for i, dep := range desc.DependedOnBy {
if dep.ID == replacingDesc.ID {
desc.DependedOnBy = append(desc.DependedOnBy[:i], desc.DependedOnBy[i+1:]...)
// TODO (lucy): Have more consistent/informative names for dependent jobs.
if err := params.p.writeSchemaChange(
params.ctx, desc, sqlbase.InvalidMutationID, "updating view reference",
); err != nil {
return err
}
break
}
}
}

// Since the view query has been replaced, the dependencies that this
// table descriptor had are gone.
replacingDesc.DependsOn = make([]sqlbase.ID, 0, len(n.planDeps))
for backrefID := range n.planDeps {
replacingDesc.DependsOn = append(replacingDesc.DependsOn, backrefID)
}

// Since we are replacing an existing view here, we need to write the new
// descriptor into place.
if err := params.p.writeSchemaChange(params.ctx, replacingDesc, sqlbase.InvalidMutationID,
fmt.Sprintf("CREATE OR REPLACE VIEW %q AS %q", n.viewName, n.viewQuery),
); err != nil {
return err
}
newDesc = replacingDesc
} else {
// If we aren't replacing anything, make a new table descriptor.
desc, err := makeViewTableDesc(
viewName,
n.viewQuery,
n.dbDesc.ID,
schemaID,
id,
n.columns,
params.creationTimeForNewTableDescriptor(),
privs,
&params.p.semaCtx,
isTemporary,
)
if err != nil {
return err
}

// Collect all the tables/views this view depends on.
for backrefID := range n.planDeps {
desc.DependsOn = append(desc.DependsOn, backrefID)
}

// TODO (lucy): I think this needs a NodeFormatter implementation. For now,
// do some basic string formatting (not accurate in the general case).
if err = params.p.createDescriptorWithID(
params.ctx, tKey.Key(), id, &desc, params.EvalContext().Settings,
fmt.Sprintf("CREATE VIEW %q AS %q", n.viewName, n.viewQuery),
); err != nil {
return err
}
newDesc = &desc
}

// Persist the back-references in all referenced table descriptors.
Expand All @@ -131,7 +251,7 @@ func (n *createViewNode) startExec(params runParams) error {
// because the ID of the newly created view descriptor was not
// yet known.
// We need to do it here.
dep.ID = desc.ID
dep.ID = newDesc.ID
backRefMutable.DependedOnBy = append(backRefMutable.DependedOnBy, dep)
}
// TODO (lucy): Have more consistent/informative names for dependent jobs.
Expand All @@ -142,7 +262,7 @@ func (n *createViewNode) startExec(params runParams) error {
}
}

if err := desc.Validate(params.ctx, params.p.txn); err != nil {
if err := newDesc.Validate(params.ctx, params.p.txn); err != nil {
return err
}

Expand All @@ -153,7 +273,7 @@ func (n *createViewNode) startExec(params runParams) error {
params.ctx,
params.p.txn,
EventLogCreateView,
int32(desc.ID),
int32(newDesc.ID),
int32(params.extendedEvalCtx.NodeID),
struct {
ViewName string
Expand Down Expand Up @@ -200,20 +320,31 @@ func makeViewTableDesc(
temporary,
)
desc.ViewQuery = viewQuery
if err := addResultColumns(semaCtx, &desc, resultColumns); err != nil {
return sqlbase.MutableTableDescriptor{}, err
}
return desc, nil
}

func addResultColumns(
semaCtx *tree.SemaContext,
desc *sqlbase.MutableTableDescriptor,
resultColumns sqlbase.ResultColumns,
) error {
for _, colRes := range resultColumns {
columnTableDef := tree.ColumnTableDef{Name: tree.Name(colRes.Name), Type: colRes.Typ}
// The new types in the CREATE VIEW column specs never use
// SERIAL so we need not process SERIAL types here.
col, _, _, err := sqlbase.MakeColumnDefDescs(&columnTableDef, semaCtx)
if err != nil {
return desc, err
return err
}
desc.AddColumn(col)
}
if err := desc.AllocateIDs(); err != nil {
return sqlbase.MutableTableDescriptor{}, err
return err
}
return desc, nil
return nil
}

func overrideColumnNames(cols sqlbase.ResultColumns, newNames tree.NameList) sqlbase.ResultColumns {
Expand Down
55 changes: 55 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/views
Expand Up @@ -567,3 +567,58 @@ SELECT * FROM v1
1 1 false
2 2 true
3 3 true

subtest create_or_replace

statement ok
DROP TABLE IF EXISTS t, t2;
CREATE TABLE t (x INT);
INSERT INTO t VALUES (1), (2);
CREATE TABLE t2 (x INT);
INSERT INTO t2 VALUES (3), (4);

# Test some error cases.

statement error pq: \"t\" is not a view
CREATE OR REPLACE VIEW t AS VALUES (1)

statement ok
CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1, x+2 AS x2 FROM t

# Test cases where new columns don't line up.

statement error pq: cannot drop columns from view
CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1 FROM t

statement error pq: cannot change name of view column \"x\" to \"xy\"
CREATE OR REPLACE VIEW tview AS SELECT x AS xy, x+1 AS x1, x+2 AS x2 FROM t

statement error pq: cannot change type of view column "x1" from int to string
CREATE OR REPLACE VIEW tview AS SELECT x AS x, (x+1)::STRING AS x1, x+2 AS x2 FROM t

statement ok
CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1, x+2 AS x2, x+3 AS x3 FROM t

query IIII rowsort
SELECT * FROM tview
----
1 2 3 4
2 3 4 5

# Test cases where back references get updated.
statement ok
CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1, x+2 AS x2, x+3 AS x3 FROM t2

query IIII rowsort
SELECT * FROM tview
----
3 4 5 6
4 5 6 7

# After remaking tview, it no longer depends on t.
statement ok
DROP TABLE t

# However, we now depend on t2.
statement error cannot drop relation "t2" because view "tview" depends on it
DROP TABLE t2
3 changes: 2 additions & 1 deletion pkg/sql/opt/bench/stub_factory.go
Expand Up @@ -383,8 +383,9 @@ func (f *stubFactory) ConstructCancelSessions(input exec.Node, ifExists bool) (e

func (f *stubFactory) ConstructCreateView(
schema cat.Schema,
viewName string,
viewName tree.TableName,
ifNotExists bool,
replace bool,
temporary bool,
viewQuery string,
columns sqlbase.ResultColumns,
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/exec/execbuilder/statement.go
Expand Up @@ -63,6 +63,7 @@ func (b *Builder) buildCreateView(cv *memo.CreateViewExpr) (execPlan, error) {
schema,
cv.ViewName,
cv.IfNotExists,
cv.Replace,
cv.Temporary,
cv.ViewQuery,
cols,
Expand Down

0 comments on commit 02c77fd

Please sign in to comment.