From 749351a7ea4c7f7c5582ef850166323e87014c8e Mon Sep 17 00:00:00 2001 From: Rohan Yadav Date: Sat, 4 Apr 2020 23:28:01 -0400 Subject: [PATCH] sql: add support for CREATE OR REPLACE VIEW 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. --- docs/generated/sql/bnf/create_view_stmt.bnf | 2 + docs/generated/sql/bnf/stmt_block.bnf | 1 + pkg/sql/create_table.go | 6 +- pkg/sql/create_view.go | 263 +++++++++++++++--- pkg/sql/logictest/testdata/logic_test/prepare | 16 ++ pkg/sql/logictest/testdata/logic_test/views | 99 +++++++ pkg/sql/opt/bench/stub_factory.go | 1 + pkg/sql/opt/exec/execbuilder/statement.go | 1 + pkg/sql/opt/exec/factory.go | 1 + pkg/sql/opt/ops/statement.opt | 2 + pkg/sql/opt/optbuilder/create_view.go | 1 + pkg/sql/opt_exec_factory.go | 2 + pkg/sql/parser/parse_test.go | 2 +- pkg/sql/parser/sql.y | 17 +- pkg/sql/sem/tree/create.go | 5 + pkg/sql/sem/tree/pretty.go | 3 + 16 files changed, 377 insertions(+), 45 deletions(-) diff --git a/docs/generated/sql/bnf/create_view_stmt.bnf b/docs/generated/sql/bnf/create_view_stmt.bnf index c9c482cbd8b1..346926d22aa1 100644 --- a/docs/generated/sql/bnf/create_view_stmt.bnf +++ b/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 diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index c619fd4fd54e..2d1ffe2b0dc3 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -1059,6 +1059,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 ::= diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 666934ed484a..81db619346c9 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -133,7 +133,8 @@ func isTypeSupportedInVersion(v clusterversion.ClusterVersion, t *types.T) bool 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) { @@ -176,7 +177,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 } diff --git a/pkg/sql/create_view.go b/pkg/sql/create_view.go index f23c5d3fdedb..7f66e634d0b5 100644 --- a/pkg/sql/create_view.go +++ b/pkg/sql/create_view.go @@ -15,7 +15,9 @@ 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/privilege" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry" @@ -30,6 +32,7 @@ type createViewNode struct { // qualified. viewQuery string ifNotExists bool + replace bool temporary bool dbDesc *sqlbase.DatabaseDescriptor columns sqlbase.ResultColumns @@ -70,12 +73,36 @@ 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 { + switch { + case !sqlbase.IsRelationAlreadyExistsError(err): + return err + case n.ifNotExists: return nil + case 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 err := params.p.CheckPrivilege(params.ctx, desc, privilege.DROP); err != nil { + return err + } + if !desc.IsView() { + return pgerror.Newf(pgcode.WrongObjectType, `%q is not a view`, viewName) + } + replacingDesc = desc + default: + return err } - return err } schemaName := tree.PublicSchemaName @@ -84,66 +111,89 @@ 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 - } - // 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, - ¶ms.p.semaCtx, - params.EvalContext(), - 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, we found an existing view while resolving + // the name for our view. So instead of creating a new view, replace + // the existing one. + if replacingDesc != nil { + newDesc, err = params.p.replaceViewDesc(params.ctx, n, replacingDesc, backRefMutables) + if err != nil { + return err + } + } else { + // If we aren't replacing anything, make a new table descriptor. + id, err := GenerateUniqueDescID(params.ctx, params.extendedEvalCtx.ExecCfg.DB) + if err != nil { + return err + } + desc, err := makeViewTableDesc( + viewName, + n.viewQuery, + n.dbDesc.ID, + schemaID, + id, + n.columns, + params.creationTimeForNewTableDescriptor(), + privs, + ¶ms.p.semaCtx, + params.p.EvalContext(), + isTemporary, + ) + if 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 + // 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. for id, updated := range n.planDeps { backRefMutable := backRefMutables[id] + // In case that we are replacing a view that already depends on + // this table, remove all existing references so that we don't leave + // any out of date references. Then, add the new references. + backRefMutable.DependedOnBy = removeMatchingReferences( + backRefMutable.DependedOnBy, + newDesc.ID, + ) for _, dep := range updated.deps { // The logical plan constructor merely registered the dependencies. // It did not populate the "ID" field of TableDescriptor_Reference, // 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. if err := params.p.writeSchemaChange( - params.ctx, backRefMutable, sqlbase.InvalidMutationID, "updating view reference", + params.ctx, + backRefMutable, + sqlbase.InvalidMutationID, + fmt.Sprintf("updating view reference %q", n.viewName), ); err != nil { return err } } - if err := desc.Validate(params.ctx, params.p.txn); err != nil { + if err := newDesc.Validate(params.ctx, params.p.txn); err != nil { return err } @@ -154,7 +204,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 @@ -202,20 +252,151 @@ func makeViewTableDesc( temporary, ) desc.ViewQuery = viewQuery + if err := addResultColumns(semaCtx, evalCtx, &desc, resultColumns); err != nil { + return sqlbase.MutableTableDescriptor{}, err + } + return desc, nil +} + +// replaceViewDesc modifies and returns the input view descriptor changed +// to hold the new view represented by n. Note that back references from +// tables that the new view depends on still need to be added. This function +// will additionally drop backreferences from tables the old view depended +// on that the new view no longer depends on. +func (p *planner) replaceViewDesc( + ctx context.Context, + n *createViewNode, + toReplace *sqlbase.MutableTableDescriptor, + backRefMutables map[sqlbase.ID]*sqlbase.MutableTableDescriptor, +) (*sqlbase.MutableTableDescriptor, error) { + // Set the query to the new query. + toReplace.ViewQuery = n.viewQuery + // Reset the columns to add the new result columns onto. + toReplace.Columns = make([]sqlbase.ColumnDescriptor, 0, len(n.columns)) + toReplace.NextColumnID = 0 + if err := addResultColumns(&p.semaCtx, p.EvalContext(), toReplace, n.columns); err != nil { + return nil, err + } + + // Compare toReplace against its ClusterVersion to verify if + // its new set of columns is valid for a replacement view. + if err := verifyReplacingViewColumns( + toReplace.ClusterVersion.Columns, + toReplace.Columns, + ); err != nil { + return nil, err + } + + // Remove the back reference from all tables that the view depended on. + for _, id := range toReplace.DependsOn { + desc, ok := backRefMutables[id] + if !ok { + var err error + desc, err = p.Tables().getMutableTableVersionByID(ctx, id, p.txn) + if err != nil { + return nil, err + } + backRefMutables[id] = desc + } + + // If n.planDeps doesn't contain id, then the new view definition doesn't + // reference this table anymore, so we can remove all existing references. + if _, ok := n.planDeps[id]; !ok { + desc.DependedOnBy = removeMatchingReferences(desc.DependedOnBy, toReplace.ID) + if err := p.writeSchemaChange( + ctx, + desc, + sqlbase.InvalidMutationID, + fmt.Sprintf("updating view reference %q", n.viewName), + ); err != nil { + return nil, err + } + } + } + + // Since the view query has been replaced, the dependencies that this + // table descriptor had are gone. + toReplace.DependsOn = make([]sqlbase.ID, 0, len(n.planDeps)) + for backrefID := range n.planDeps { + toReplace.DependsOn = append(toReplace.DependsOn, backrefID) + } + + // Since we are replacing an existing view here, we need to write the new + // descriptor into place. + if err := p.writeSchemaChange(ctx, toReplace, sqlbase.InvalidMutationID, + fmt.Sprintf("CREATE OR REPLACE VIEW %q AS %q", n.viewName, n.viewQuery), + ); err != nil { + return nil, err + } + return toReplace, nil +} + +// addResultColumns adds the resultColumns as actual column +// descriptors onto desc. +func addResultColumns( + semaCtx *tree.SemaContext, + evalCtx *tree.EvalContext, + 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, evalCtx) 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 +} + +// verifyReplacingViewColumns ensures that the new set of view columns 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. +func verifyReplacingViewColumns(oldColumns, newColumns []sqlbase.ColumnDescriptor) error { + if len(newColumns) < len(oldColumns) { + return pgerror.Newf(pgcode.InvalidTableDefinition, "cannot drop columns from view") + } + for i := range oldColumns { + oldCol, newCol := &oldColumns[i], &newColumns[i] + if oldCol.Name != newCol.Name { + return pgerror.Newf( + pgcode.InvalidTableDefinition, + `cannot change name of view column %q to %q`, + oldCol.Name, + newCol.Name, + ) + } + if !newCol.Type.Equal(oldCol.Type) { + return pgerror.Newf( + pgcode.InvalidTableDefinition, + `cannot change type of view column %q from %s to %s`, + oldCol.Name, + oldCol.Type.String(), + newCol.Type.String(), + ) + } + if newCol.Hidden != oldCol.Hidden { + return pgerror.Newf( + pgcode.InvalidTableDefinition, + `cannot change visibility of view column %q`, + oldCol.Name, + ) + } + if newCol.Nullable != oldCol.Nullable { + return pgerror.Newf( + pgcode.InvalidTableDefinition, + `cannot change nullability of view column %q`, + oldCol.Name, + ) + } + } + return nil } func overrideColumnNames(cols sqlbase.ResultColumns, newNames tree.NameList) sqlbase.ResultColumns { diff --git a/pkg/sql/logictest/testdata/logic_test/prepare b/pkg/sql/logictest/testdata/logic_test/prepare index 5d35797cca44..fe77d82e33e6 100644 --- a/pkg/sql/logictest/testdata/logic_test/prepare +++ b/pkg/sql/logictest/testdata/logic_test/prepare @@ -1232,3 +1232,19 @@ PREPARE q AS DELETE FROM ts WHERE ts.d <= $1 statement ok EXECUTE q ('2020-03-25') + +# Test that if we replace a view the cached plan is invalidated. +statement ok +CREATE VIEW tview AS VALUES (1) + +statement ok +PREPARE tview_prep AS SELECT * FROM tview + +statement ok +CREATE OR REPLACE VIEW tview AS VALUES (2) + +query I +EXECUTE tview_prep +---- +2 + diff --git a/pkg/sql/logictest/testdata/logic_test/views b/pkg/sql/logictest/testdata/logic_test/views index 8d4259f2d6c1..03703a54767f 100644 --- a/pkg/sql/logictest/testdata/logic_test/views +++ b/pkg/sql/logictest/testdata/logic_test/views @@ -567,3 +567,102 @@ SELECT * FROM v1 1 1 false 2 2 true 3 3 true + +subtest create_or_replace + +user root + +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 + +# Test that if we add a reference to something in t2 and use it when replacing +# the view that we now reference that object as well. +statement ok +CREATE INDEX i ON t2 (x); +CREATE INDEX i2 ON t2 (x); +CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1, x+2 AS x2, x+3 AS x3 FROM t2@i + +statement error pq: cannot drop index \"i\" because view \"tview\" depends on it +DROP INDEX t2@i + +# However, if we change the view, we should be able to drop i. +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@i2; +DROP INDEX t2@i + +# ... and not i2. +statement error pq: cannot drop index \"i2\" because view \"tview\" depends on it +DROP INDEX t2@i2 + +# Ensure that users can't replace views they don't have privilege to. +statement ok +GRANT CREATE ON DATABASE test TO testuser; +GRANT CREATE, SELECT ON TABLE tview, t2 TO testuser + +user testuser + +statement error pq: user testuser does not have DROP privilege on relation tview +CREATE OR REPLACE VIEW tview AS SELECT x AS x, x+1 AS x1, x+2 AS x2, x+3 AS x3 FROM t2 + +# Give privilege now. +user root + +statement ok +GRANT DROP ON TABLE tview TO testuser + +user testuser + +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 + +user root diff --git a/pkg/sql/opt/bench/stub_factory.go b/pkg/sql/opt/bench/stub_factory.go index 6f00a6d34d1f..059e619dc6aa 100644 --- a/pkg/sql/opt/bench/stub_factory.go +++ b/pkg/sql/opt/bench/stub_factory.go @@ -385,6 +385,7 @@ func (f *stubFactory) ConstructCreateView( schema cat.Schema, viewName string, ifNotExists bool, + replace bool, temporary bool, viewQuery string, columns sqlbase.ResultColumns, diff --git a/pkg/sql/opt/exec/execbuilder/statement.go b/pkg/sql/opt/exec/execbuilder/statement.go index b1b6486bda55..a576131fdc08 100644 --- a/pkg/sql/opt/exec/execbuilder/statement.go +++ b/pkg/sql/opt/exec/execbuilder/statement.go @@ -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, diff --git a/pkg/sql/opt/exec/factory.go b/pkg/sql/opt/exec/factory.go index 7ce61a4d9e7a..43ba3f6d6d5f 100644 --- a/pkg/sql/opt/exec/factory.go +++ b/pkg/sql/opt/exec/factory.go @@ -484,6 +484,7 @@ type Factory interface { schema cat.Schema, viewName string, ifNotExists bool, + replace bool, temporary bool, viewQuery string, columns sqlbase.ResultColumns, diff --git a/pkg/sql/opt/ops/statement.opt b/pkg/sql/opt/ops/statement.opt index 9c4731f1bb83..5af903f280eb 100644 --- a/pkg/sql/opt/ops/statement.opt +++ b/pkg/sql/opt/ops/statement.opt @@ -45,6 +45,8 @@ define CreateViewPrivate { IfNotExists bool + Replace bool + # ViewQuery contains the query for the view; data sources are always fully # qualified. ViewQuery string diff --git a/pkg/sql/opt/optbuilder/create_view.go b/pkg/sql/opt/optbuilder/create_view.go index f1414935044c..70133f285a0e 100644 --- a/pkg/sql/opt/optbuilder/create_view.go +++ b/pkg/sql/opt/optbuilder/create_view.go @@ -64,6 +64,7 @@ func (b *Builder) buildCreateView(cv *tree.CreateView, inScope *scope) (outScope Schema: schID, ViewName: cv.Name.Table(), IfNotExists: cv.IfNotExists, + Replace: cv.Replace, Temporary: cv.Temporary, ViewQuery: tree.AsStringWithFlags(cv.AsSource, tree.FmtParsable), Columns: p, diff --git a/pkg/sql/opt_exec_factory.go b/pkg/sql/opt_exec_factory.go index f765be299cd6..de166d87f03a 100644 --- a/pkg/sql/opt_exec_factory.go +++ b/pkg/sql/opt_exec_factory.go @@ -1712,6 +1712,7 @@ func (ef *execFactory) ConstructCreateView( schema cat.Schema, viewName string, ifNotExists bool, + replace bool, temporary bool, viewQuery string, columns sqlbase.ResultColumns, @@ -1744,6 +1745,7 @@ func (ef *execFactory) ConstructCreateView( return &createViewNode{ viewName: tree.Name(viewName), ifNotExists: ifNotExists, + replace: replace, temporary: temporary, viewQuery: viewQuery, dbDesc: schema.(*optSchema).desc, diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index abad80622680..101ef8885287 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -293,6 +293,7 @@ func TestParse(t *testing.T) { {`CREATE TABLE a (LIKE b INCLUDING ALL EXCLUDING INDEXES, c INT8)`}, {`CREATE VIEW a AS SELECT * FROM b`}, + {`CREATE OR REPLACE VIEW a AS SELECT * FROM b`}, {`EXPLAIN CREATE VIEW a AS SELECT * FROM b`}, {`CREATE VIEW a AS SELECT b.* FROM b LIMIT 5`}, {`CREATE VIEW a AS (SELECT c, d FROM b WHERE c > 0 ORDER BY c)`}, @@ -3238,7 +3239,6 @@ func TestUnimplementedSyntax(t *testing.T) { {`CREATE SEQUENCE a AS DOUBLE PRECISION`, 25110, `FLOAT8`, ``}, - {`CREATE OR REPLACE VIEW a AS SELECT b`, 24897, ``, ``}, {`CREATE RECURSIVE VIEW a AS SELECT b`, 0, `create recursive view`, ``}, {`CREATE TYPE a AS (b)`, 27792, ``, ``}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index e593ef2058b8..b2c152c17fd8 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -5327,6 +5327,21 @@ create_view_stmt: AsSource: $8.slct(), Temporary: $2.persistenceType(), IfNotExists: false, + Replace: false, + } + } +// We cannot use a rule like opt_or_replace here as that would cause a conflict +// with the opt_temp rule. +| CREATE OR REPLACE opt_temp opt_view_recursive VIEW view_name opt_column_list AS select_stmt + { + name := $7.unresolvedObjectName().ToTableName() + $$.val = &tree.CreateView{ + Name: name, + ColumnNames: $8.nameList(), + AsSource: $10.slct(), + Temporary: $4.persistenceType(), + IfNotExists: false, + Replace: true, } } | CREATE opt_temp opt_view_recursive VIEW IF NOT EXISTS view_name opt_column_list AS select_stmt @@ -5338,9 +5353,9 @@ create_view_stmt: AsSource: $11.slct(), Temporary: $2.persistenceType(), IfNotExists: true, + Replace: false, } } -| CREATE OR REPLACE opt_temp opt_view_recursive VIEW error { return unimplementedWithIssue(sqllex, 24897) } | CREATE opt_temp opt_view_recursive VIEW error // SHOW HELP: CREATE VIEW role_option: diff --git a/pkg/sql/sem/tree/create.go b/pkg/sql/sem/tree/create.go index 660e1109466e..46be76113c9b 100644 --- a/pkg/sql/sem/tree/create.go +++ b/pkg/sql/sem/tree/create.go @@ -1445,12 +1445,17 @@ type CreateView struct { AsSource *Select IfNotExists bool Temporary bool + Replace bool } // Format implements the NodeFormatter interface. func (node *CreateView) Format(ctx *FmtCtx) { ctx.WriteString("CREATE ") + if node.Replace { + ctx.WriteString("OR REPLACE ") + } + if node.Temporary { ctx.WriteString("TEMPORARY ") } diff --git a/pkg/sql/sem/tree/pretty.go b/pkg/sql/sem/tree/pretty.go index 37d52dcca0c4..84372818cd03 100644 --- a/pkg/sql/sem/tree/pretty.go +++ b/pkg/sql/sem/tree/pretty.go @@ -1229,6 +1229,9 @@ func (node *CreateView) doc(p *PrettyCfg) pretty.Doc { // SELECT ... // title := pretty.Keyword("CREATE") + if node.Replace { + title = pretty.ConcatSpace(title, pretty.Keyword("OR REPLACE")) + } if node.Temporary { title = pretty.ConcatSpace(title, pretty.Keyword("TEMPORARY")) }