diff --git a/docs/generated/sql/bnf/create_table_as_stmt.bnf b/docs/generated/sql/bnf/create_table_as_stmt.bnf index ca24d7c3ab1d..ce493ace6657 100644 --- a/docs/generated/sql/bnf/create_table_as_stmt.bnf +++ b/docs/generated/sql/bnf/create_table_as_stmt.bnf @@ -1,5 +1,3 @@ create_table_as_stmt ::= - 'CREATE' 'TABLE' table_name '(' name ( ( ',' name ) )* ')' 'AS' select_stmt - | 'CREATE' 'TABLE' table_name 'AS' select_stmt - | 'CREATE' 'TABLE' 'IF' 'NOT' 'EXISTS' table_name '(' name ( ( ',' name ) )* ')' 'AS' select_stmt - | 'CREATE' 'TABLE' 'IF' 'NOT' 'EXISTS' table_name 'AS' select_stmt + 'CREATE' 'TABLE' table_name create_as_opt_col_list 'AS' select_stmt + | 'CREATE' 'TABLE' 'IF' 'NOT' 'EXISTS' table_name create_as_opt_col_list 'AS' select_stmt diff --git a/docs/generated/sql/bnf/stmt_block.bnf b/docs/generated/sql/bnf/stmt_block.bnf index 6fbbc5e914f7..c69211f57493 100644 --- a/docs/generated/sql/bnf/stmt_block.bnf +++ b/docs/generated/sql/bnf/stmt_block.bnf @@ -969,8 +969,8 @@ create_table_stmt ::= | 'CREATE' 'TABLE' 'IF' 'NOT' 'EXISTS' table_name '(' opt_table_elem_list ')' opt_interleave opt_partition_by create_table_as_stmt ::= - 'CREATE' 'TABLE' table_name opt_column_list 'AS' select_stmt - | 'CREATE' 'TABLE' 'IF' 'NOT' 'EXISTS' table_name opt_column_list 'AS' select_stmt + 'CREATE' 'TABLE' table_name create_as_opt_col_list 'AS' select_stmt + | 'CREATE' 'TABLE' 'IF' 'NOT' 'EXISTS' table_name create_as_opt_col_list 'AS' select_stmt create_view_stmt ::= 'CREATE' 'VIEW' view_name opt_column_list 'AS' select_stmt @@ -1352,6 +1352,10 @@ opt_table_elem_list ::= table_elem_list | +create_as_opt_col_list ::= + '(' create_as_table_defs ')' + | + view_name ::= table_name @@ -1689,6 +1693,9 @@ partition_by ::= | 'PARTITION' 'BY' 'RANGE' '(' name_list ')' '(' range_partitions ')' | 'PARTITION' 'BY' 'NOTHING' +create_as_table_defs ::= + ( column_name create_as_col_qual_list ) ( ( ',' create_as_constraint_def | ',' column_name create_as_col_qual_list ) )* + common_table_expr ::= table_alias_name opt_column_list 'AS' '(' preparable_stmt ')' @@ -1911,6 +1918,12 @@ list_partitions ::= range_partitions ::= ( range_partition ) ( ( ',' range_partition ) )* +create_as_col_qual_list ::= + ( ) ( ( create_as_col_qualification ) )* + +create_as_constraint_def ::= + create_as_constraint_elem + index_flags_param ::= 'FORCE_INDEX' '=' index_name | 'NO_INDEX_JOIN' @@ -2121,6 +2134,12 @@ list_partition ::= range_partition ::= partition 'VALUES' 'FROM' '(' expr_list ')' 'TO' '(' expr_list ')' opt_partition_by +create_as_col_qualification ::= + create_as_col_qualification_elem + +create_as_constraint_elem ::= + 'PRIMARY' 'KEY' '(' create_as_params ')' + col_qualification_elem ::= 'NOT' 'NULL' | 'NULL' @@ -2230,6 +2249,12 @@ func_expr_windowless ::= rowsfrom_list ::= ( rowsfrom_item ) ( ( ',' rowsfrom_item ) )* +create_as_col_qualification_elem ::= + 'PRIMARY' 'KEY' + +create_as_params ::= + ( create_as_param ) ( ( ',' create_as_param ) )* + opt_name_parens ::= '(' name ')' | @@ -2295,6 +2320,9 @@ join_outer ::= rowsfrom_item ::= func_expr_windowless +create_as_param ::= + column_name + frame_extent ::= frame_bound | 'BETWEEN' frame_bound 'AND' frame_bound diff --git a/pkg/sql/create_table.go b/pkg/sql/create_table.go index 3bd035b81593..c8afae09dd3a 100644 --- a/pkg/sql/create_table.go +++ b/pkg/sql/create_table.go @@ -69,7 +69,12 @@ func (p *planner) CreateTable(ctx context.Context, n *tree.CreateTable) (planNod return nil, err } - numColNames := len(n.AsColumnNames) + numColNames := 0 + for i := 0; i < len(n.Defs); i++ { + if _, ok := n.Defs[i].(*tree.ColumnTableDef); ok { + numColNames++ + } + } numColumns := len(planColumns(sourcePlan)) if numColNames != 0 && numColNames != numColumns { sourcePlan.Close(ctx) @@ -80,12 +85,22 @@ func (p *planner) CreateTable(ctx context.Context, n *tree.CreateTable) (planNod } // Synthesize an input column that provides the default value for the - // hidden rowid column. + // hidden rowid column, if none of the provided columns are specified + // as the PRIMARY KEY. synthRowID = true + for _, def := range n.Defs { + if d, ok := def.(*tree.ColumnTableDef); ok && d.PrimaryKey { + synthRowID = false + break + } + } } ct := &createTableNode{n: n, dbDesc: dbDesc, sourcePlan: sourcePlan} ct.run.synthRowID = synthRowID + // This method is only invoked if the heuristic planner was used in the + // planning stage. + ct.run.fromHeuristicPlanner = true return ct, nil } @@ -95,10 +110,17 @@ type createTableRun struct { autoCommit autoCommitOpt // synthRowID indicates whether an input column needs to be synthesized to - // provide the default value for the hidden rowid column. The optimizer's - // plan already includes this column (so synthRowID is false), whereas the - // heuristic planner's plan does not (so synthRowID is true). + // provide the default value for the hidden rowid column. The optimizer's plan + // already includes this column if a user specified PK does not exist (so + // synthRowID is false), whereas the heuristic planner's plan does not in this + // case (so synthRowID is true). synthRowID bool + + // fromHeuristicPlanner indicates whether the planning was performed by the + // heuristic planner instead of the optimizer. This is used to determine + // whether or not a row_id was synthesized as part of the planning stage, if a + // user defined PK is not specified. + fromHeuristicPlanner bool } func (n *createTableNode) startExec(params runParams) error { @@ -139,15 +161,18 @@ func (n *createTableNode) startExec(params runParams) error { } asCols = planColumns(n.sourcePlan) - if !n.run.synthRowID { - // rowID column is already present in the input as the last column, so - // ignore it for the purpose of creating column metadata (because + if !n.run.fromHeuristicPlanner && !n.n.AsHasUserSpecifiedPrimaryKey() { + // rowID column is already present in the input as the last column if it + // was planned by the optimizer and the user did not specify a PRIMARY + // KEY. So ignore it for the purpose of creating column metadata (because // makeTableDescIfAs does it automatically). asCols = asCols[:len(asCols)-1] } - desc, err = makeTableDescIfAs( + + desc, err = makeTableDescIfAs(params, n.n, n.dbDesc.ID, id, creationTime, asCols, - privs, ¶ms.p.semaCtx, params.p.EvalContext()) + privs, params.p.EvalContext()) + if err != nil { return err } @@ -259,9 +284,9 @@ func (n *createTableNode) startExec(params runParams) error { return err } - // Prepare the buffer for row values. At this point, one more - // column has been added by ensurePrimaryKey() to the list of - // columns in sourcePlan. + // Prepare the buffer for row values. At this point, one more column has + // been added by ensurePrimaryKey() to the list of columns in sourcePlan, if + // a PRIMARY KEY is not specified by the user. rowBuffer := make(tree.Datums, len(desc.Columns)) pkColIdx := len(desc.Columns) - 1 @@ -985,38 +1010,51 @@ func getFinalSourceQuery(source *tree.Select, evalCtx *tree.EvalContext) string // makeTableDescIfAs is the MakeTableDesc method for when we have a table // that is created with the CREATE AS format. func makeTableDescIfAs( + params runParams, p *tree.CreateTable, parentID, id sqlbase.ID, creationTime hlc.Timestamp, resultColumns []sqlbase.ResultColumn, privileges *sqlbase.PrivilegeDescriptor, - semaCtx *tree.SemaContext, evalContext *tree.EvalContext, ) (desc sqlbase.MutableTableDescriptor, err error) { - desc = InitTableDescriptor(id, parentID, p.Table.Table(), creationTime, privileges) - desc.CreateQuery = getFinalSourceQuery(p.AsSource, evalContext) - - for i, colRes := range resultColumns { - columnTableDef := tree.ColumnTableDef{Name: tree.Name(colRes.Name), Type: colRes.Typ} - columnTableDef.Nullable.Nullability = tree.SilentNull - if len(p.AsColumnNames) > i { - columnTableDef.Name = p.AsColumnNames[i] - } - - // The new types in the CREATE TABLE AS 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 + colResIndex := 0 + // TableDefs for a CREATE TABLE ... AS AST node comprise of a ColumnTableDef + // for each column, and a ConstraintTableDef for any constraints on those + // columns. + for _, defs := range p.Defs { + var d *tree.ColumnTableDef + var ok bool + if d, ok = defs.(*tree.ColumnTableDef); ok { + d.Type = resultColumns[colResIndex].Typ + colResIndex++ + } + } + + // If there are no TableDefs defined by the parser, then we construct a + // ColumnTableDef for each column using resultColumns. + if len(p.Defs) == 0 { + for _, colRes := range resultColumns { + var d *tree.ColumnTableDef + var ok bool + var tableDef tree.TableDef = &tree.ColumnTableDef{Name: tree.Name(colRes.Name), Type: colRes.Typ} + if d, ok = tableDef.(*tree.ColumnTableDef); !ok { + return desc, errors.Errorf("failed to cast type to ColumnTableDef\n") + } + d.Nullable.Nullability = tree.SilentNull + p.Defs = append(p.Defs, tableDef) } - desc.AddColumn(col) } - // AllocateIDs mutates its receiver. `return desc, desc.AllocateIDs()` - // happens to work in gc, but does not work in gccgo. - // - // See https://github.com/golang/go/issues/23188. - err = desc.AllocateIDs() + desc, err = makeTableDesc( + params, + p, + parentID, id, + creationTime, + privileges, + nil, /* affected */ + ) + desc.CreateQuery = getFinalSourceQuery(p.AsSource, evalContext) return desc, err } diff --git a/pkg/sql/logictest/testdata/logic_test/create_as b/pkg/sql/logictest/testdata/logic_test/create_as index 4f70a8146e75..d43a40765d37 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_as +++ b/pkg/sql/logictest/testdata/logic_test/create_as @@ -144,3 +144,95 @@ SELECT * FROM baz ---- a b c 1 2 4 + +# Check that CREATE TABLE AS allows users to specify primary key (#20940) +statement ok +CREATE TABLE foo5 (a, b PRIMARY KEY, c) AS SELECT * FROM baz + +query TT +SHOW CREATE TABLE foo5 +---- +foo5 CREATE TABLE foo5 ( + a INT8 NULL, + b INT8 NOT NULL, + c INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (b ASC), + FAMILY "primary" (a, b, c) + ) + +statement ok +SET OPTIMIZER=ON; CREATE TABLE foo6 (a PRIMARY KEY, b, c) AS SELECT * FROM baz; SET OPTIMIZER=OFF + +query TT +SHOW CREATE TABLE foo6 +---- +foo6 CREATE TABLE foo6 ( + a INT8 NOT NULL, + b INT8 NULL, + c INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (a ASC), + FAMILY "primary" (a, b, c) + ) + +statement error generate insert row: null value in column "x" violates not-null constraint +CREATE TABLE foo7 (x PRIMARY KEY) AS VALUES (1), (NULL); + +statement ok +BEGIN; CREATE TABLE foo8 (item PRIMARY KEY, qty) AS SELECT * FROM stock UNION VALUES ('spoons', 25), ('knives', 50); END + +query TT +SHOW CREATE TABLE foo8 +---- +foo8 CREATE TABLE foo8 ( + item STRING NOT NULL, + qty INT8 NULL, + CONSTRAINT "primary" PRIMARY KEY (item ASC), + FAMILY "primary" (item, qty) + ) + +# Allow CREATE TABLE AS to specify composite primary keys. +statement ok +CREATE TABLE foo9 (a, b, c, PRIMARY KEY (a, c)) AS SELECT * FROM baz + +query TT +SHOW CREATE TABLE foo9 +---- +foo9 CREATE TABLE foo9 ( + a INT8 NOT NULL, + b INT8 NULL, + c INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (a ASC, c ASC), + FAMILY "primary" (a, b, c) + ) + +statement ok +CREATE TABLE foo10 (a, PRIMARY KEY (c, b, a), b, c) AS SELECT * FROM foo9 + +query TT +SHOW CREATE TABLE foo10 +---- +foo10 CREATE TABLE foo10 ( + a INT8 NOT NULL, + b INT8 NOT NULL, + c INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (c ASC, b ASC, a ASC), + FAMILY "primary" (a, b, c) + ) + +statement ok +CREATE TABLE foo11 (x, y, z, PRIMARY KEY(x, z)) AS VALUES (1, 3, 4), (10, 20, 40); + +query TT +SHOW CREATE TABLE foo11 +---- +foo11 CREATE TABLE foo11 ( + x INT8 NOT NULL, + y INT8 NULL, + z INT8 NOT NULL, + CONSTRAINT "primary" PRIMARY KEY (x ASC, z ASC), + FAMILY "primary" (x, y, z) + ) + +statement error pq: multiple primary keys for table "foo12" are not allowed +CREATE TABLE foo12 (x PRIMARY KEY, y, PRIMARY KEY(y)) AS VALUES (1, 2), (3, 4); + diff --git a/pkg/sql/opt/optbuilder/create_table.go b/pkg/sql/opt/optbuilder/create_table.go index 8cc942bff0c3..aa57e827598f 100644 --- a/pkg/sql/opt/optbuilder/create_table.go +++ b/pkg/sql/opt/optbuilder/create_table.go @@ -42,7 +42,12 @@ func (b *Builder) buildCreateTable(ct *tree.CreateTable, inScope *scope) (outSco // Build the input query. outScope := b.buildSelect(ct.AsSource, nil /* desiredTypes */, inScope) - numColNames := len(ct.AsColumnNames) + numColNames := 0 + for i := 0; i < len(ct.Defs); i++ { + if _, ok := ct.Defs[i].(*tree.ColumnTableDef); ok { + numColNames++ + } + } numColumns := len(outScope.cols) if numColNames != 0 && numColNames != numColumns { panic(sqlbase.NewSyntaxError(fmt.Sprintf( @@ -51,17 +56,20 @@ func (b *Builder) buildCreateTable(ct *tree.CreateTable, inScope *scope) (outSco numColumns, util.Pluralize(int64(numColumns))))) } - // Synthesize rowid column, and append to end of column list. - props, overloads := builtins.GetBuiltinProperties("unique_rowid") - private := &memo.FunctionPrivate{ - Name: "unique_rowid", - Typ: types.Int, - Properties: props, - Overload: &overloads[0], + input = outScope.expr + if !ct.AsHasUserSpecifiedPrimaryKey() { + // Synthesize rowid column, and append to end of column list. + props, overloads := builtins.GetBuiltinProperties("unique_rowid") + private := &memo.FunctionPrivate{ + Name: "unique_rowid", + Typ: types.Int, + Properties: props, + Overload: &overloads[0], + } + fn := b.factory.ConstructFunction(memo.EmptyScalarListExpr, private) + scopeCol := b.synthesizeColumn(outScope, "rowid", types.Int, nil /* expr */, fn) + input = b.factory.CustomFuncs().ProjectExtraCol(outScope.expr, fn, scopeCol.id) } - fn := b.factory.ConstructFunction(memo.EmptyScalarListExpr, private) - scopeCol := b.synthesizeColumn(outScope, "rowid", types.Int, nil /* expr */, fn) - input = b.factory.CustomFuncs().ProjectExtraCol(outScope.expr, fn, scopeCol.id) inputCols = outScope.makePhysicalProps().Presentation } else { // Create dummy empty input. diff --git a/pkg/sql/parser/parse_test.go b/pkg/sql/parser/parse_test.go index 1ff283f187fd..372af167edaa 100644 --- a/pkg/sql/parser/parse_test.go +++ b/pkg/sql/parser/parse_test.go @@ -253,6 +253,10 @@ func TestParse(t *testing.T) { {`CREATE TABLE IF NOT EXISTS a AS SELECT * FROM b UNION SELECT * FROM c`}, {`CREATE TABLE a AS SELECT * FROM b UNION VALUES ('one', 1) ORDER BY c LIMIT 5`}, {`CREATE TABLE IF NOT EXISTS a AS SELECT * FROM b UNION VALUES ('one', 1) ORDER BY c LIMIT 5`}, + {`CREATE TABLE a (z PRIMARY KEY) AS SELECT * FROM b`}, + {`CREATE TABLE IF NOT EXISTS a (z PRIMARY KEY) AS SELECT * FROM b`}, + {`CREATE TABLE a (x, y, z, PRIMARY KEY (x, y, z)) AS SELECT * FROM b`}, + {`CREATE TABLE IF NOT EXISTS a (x, y, z, PRIMARY KEY (x, y, z)) AS SELECT * FROM b`}, {`CREATE TABLE a (b STRING COLLATE de)`}, {`CREATE TABLE a (b STRING(3) COLLATE de)`}, {`CREATE TABLE a (b STRING[] COLLATE de)`}, diff --git a/pkg/sql/parser/sql.y b/pkg/sql/parser/sql.y index 8bdd43a095ee..fea8b3edceb1 100644 --- a/pkg/sql/parser/sql.y +++ b/pkg/sql/parser/sql.y @@ -802,7 +802,7 @@ func newNameFromStr(s string) *tree.Name { %type iso_level %type user_priority -%type opt_table_elem_list table_elem_list +%type opt_table_elem_list table_elem_list create_as_opt_col_list create_as_table_defs %type <*tree.InterleaveDef> opt_interleave %type <*tree.PartitionBy> opt_partition_by partition_by %type partition opt_partition @@ -816,7 +816,7 @@ func newNameFromStr(s string) *tree.Name { %type opt_column_list insert_column_list opt_stats_columns %type sort_clause opt_sort_clause %type <[]*tree.Order> sortby_list -%type index_params +%type index_params create_as_params %type name_list privilege_list %type <[]int32> opt_array_bounds %type <*tree.From> from_clause update_from_clause @@ -896,7 +896,7 @@ func newNameFromStr(s string) *tree.Name { %type alias_clause opt_alias_clause %type opt_ordinality opt_compact opt_automatic %type <*tree.Order> sortby -%type index_elem +%type index_elem create_as_param %type table_ref func_table %type rowsfrom_list %type rowsfrom_item @@ -941,12 +941,12 @@ func newNameFromStr(s string) *tree.Name { %type unreserved_keyword type_func_name_keyword cockroachdb_extra_type_func_name_keyword %type col_name_keyword reserved_keyword cockroachdb_extra_reserved_keyword extra_var_value -%type table_constraint constraint_elem +%type table_constraint constraint_elem create_as_constraint_def create_as_constraint_elem %type index_def %type family_def -%type <[]tree.NamedColumnQualification> col_qual_list -%type col_qualification -%type col_qualification_elem +%type <[]tree.NamedColumnQualification> col_qual_list create_as_col_qual_list +%type col_qualification create_as_col_qualification +%type col_qualification_elem create_as_col_qualification_elem %type key_match %type reference_actions %type reference_action reference_on_delete reference_on_update @@ -3997,7 +3997,6 @@ create_table_stmt: Interleave: $8.interleave(), Defs: $6.tblDefs(), AsSource: nil, - AsColumnNames: nil, PartitionBy: $9.partitionBy(), } } @@ -4010,7 +4009,6 @@ create_table_stmt: Interleave: $11.interleave(), Defs: $9.tblDefs(), AsSource: nil, - AsColumnNames: nil, PartitionBy: $12.partitionBy(), } } @@ -4021,28 +4019,26 @@ opt_table_with: | WITH name error { return unimplemented(sqllex, "create table with " + $2) } create_table_as_stmt: - CREATE opt_temp TABLE table_name opt_column_list opt_table_with AS select_stmt opt_create_as_data + CREATE opt_temp TABLE table_name create_as_opt_col_list opt_table_with AS select_stmt opt_create_as_data { name := $4.unresolvedObjectName().ToTableName() $$.val = &tree.CreateTable{ Table: name, IfNotExists: false, Interleave: nil, - Defs: nil, + Defs: $5.tblDefs(), AsSource: $8.slct(), - AsColumnNames: $5.nameList(), } } -| CREATE opt_temp TABLE IF NOT EXISTS table_name opt_column_list opt_table_with AS select_stmt opt_create_as_data +| CREATE opt_temp TABLE IF NOT EXISTS table_name create_as_opt_col_list opt_table_with AS select_stmt opt_create_as_data { name := $7.unresolvedObjectName().ToTableName() $$.val = &tree.CreateTable{ Table: name, IfNotExists: true, Interleave: nil, - Defs: nil, + Defs: $8.tblDefs(), AsSource: $11.slct(), - AsColumnNames: $8.nameList(), } } @@ -4425,6 +4421,100 @@ constraint_elem: } } + +create_as_opt_col_list: + '(' create_as_table_defs ')' + { + $$.val = $2.val + } +| /* EMPTY */ + { + $$.val = tree.TableDefs(nil) + } + +create_as_table_defs: + column_name create_as_col_qual_list + { + tableDef, err := tree.NewColumnTableDef(tree.Name($1), nil, false, $2.colQuals()) + if err != nil { + return setErr(sqllex, err) + } + + var colToTableDef tree.TableDef = tableDef + $$.val = tree.TableDefs{colToTableDef} + } +| create_as_table_defs ',' create_as_constraint_def + { + var constraintToTableDef tree.TableDef = $3.constraintDef() + $$.val = append($1.tblDefs(), constraintToTableDef) + } +| create_as_table_defs ',' column_name create_as_col_qual_list + { + tableDef, err := tree.NewColumnTableDef(tree.Name($3), nil, false, $4.colQuals()) + if err != nil { + return setErr(sqllex, err) + } + + var colToTableDef tree.TableDef = tableDef + + $$.val = append($1.tblDefs(), colToTableDef) + } + +create_as_constraint_def: + create_as_constraint_elem + { + $$.val = $1.constraintDef() + } + +create_as_constraint_elem: + PRIMARY KEY '(' create_as_params ')' + { + $$.val = &tree.UniqueConstraintTableDef{ + IndexTableDef: tree.IndexTableDef{ + Columns: $4.idxElems(), + }, + PrimaryKey: true, + } + } + +create_as_params: + create_as_param + { + $$.val = tree.IndexElemList{$1.idxElem()} + } +| create_as_params ',' create_as_param + { + $$.val = append($1.idxElems(), $3.idxElem()) + } + +create_as_param: + column_name + { + $$.val = tree.IndexElem{Column: tree.Name($1)} + } + +create_as_col_qual_list: + create_as_col_qual_list create_as_col_qualification + { + $$.val = append($1.colQuals(), $2.colQual()) + } +| /* EMPTY */ + { + $$.val = []tree.NamedColumnQualification(nil) + } + +create_as_col_qualification: + create_as_col_qualification_elem + { + $$.val = tree.NamedColumnQualification{Qualification: $1.colQualElem()} + } + +create_as_col_qualification_elem: + PRIMARY KEY + { + $$.val = tree.PrimaryKeyConstraint{} + } + opt_deferrable: /* EMPTY */ { /* no error */ } | DEFERRABLE { return unimplementedWithIssueDetail(sqllex, 31632, "deferrable") } diff --git a/pkg/sql/sem/tree/create.go b/pkg/sql/sem/tree/create.go index f37c40c8ac96..d709022f4cae 100644 --- a/pkg/sql/sem/tree/create.go +++ b/pkg/sql/sem/tree/create.go @@ -358,8 +358,14 @@ func (node *ColumnTableDef) HasColumnFamily() bool { // Format implements the NodeFormatter interface. func (node *ColumnTableDef) Format(ctx *FmtCtx) { ctx.FormatNode(&node.Name) - ctx.WriteByte(' ') - ctx.WriteString(node.columnTypeString()) + + // ColumnTableDef node type will not be specified if it represents a CREATE + // TABLE ... AS query. + if node.Type != nil { + ctx.WriteByte(' ') + ctx.WriteString(node.columnTypeString()) + } + if node.Nullable.Nullability != SilentNull && node.Nullable.ConstraintName != "" { ctx.WriteString(" CONSTRAINT ") ctx.FormatNode(&node.Nullable.ConstraintName) @@ -883,13 +889,15 @@ func (node *RangePartition) Format(ctx *FmtCtx) { // CreateTable represents a CREATE TABLE statement. type CreateTable struct { - IfNotExists bool - Table TableName - Interleave *InterleaveDef - PartitionBy *PartitionBy - Defs TableDefs - AsSource *Select - AsColumnNames NameList // Only to be used in conjunction with AsSource + IfNotExists bool + Table TableName + Interleave *InterleaveDef + PartitionBy *PartitionBy + // In CREATE...AS queries, Defs represents a list of ColumnTableDefs, one for + // each column, and a ConstraintTableDef for each constraint on a subset of + // these columns. + Defs TableDefs + AsSource *Select } // As returns true if this table represents a CREATE TABLE ... AS statement, @@ -898,6 +906,21 @@ func (node *CreateTable) As() bool { return node.AsSource != nil } +// AsHasUserSpecifiedPrimaryKey returns true if a CREATE TABLE ... AS statement +// has a PRIMARY KEY constraint specified. +func (node *CreateTable) AsHasUserSpecifiedPrimaryKey() bool { + if node.As() { + for _, def := range node.Defs { + if d, ok := def.(*ColumnTableDef); !ok { + return false + } else if d.PrimaryKey { + return true + } + } + } + return false +} + // Format implements the NodeFormatter interface. func (node *CreateTable) Format(ctx *FmtCtx) { ctx.WriteString("CREATE TABLE ") @@ -912,9 +935,9 @@ func (node *CreateTable) Format(ctx *FmtCtx) { // but the CREATE TABLE tableName part. func (node *CreateTable) FormatBody(ctx *FmtCtx) { if node.As() { - if len(node.AsColumnNames) > 0 { + if len(node.Defs) > 0 { ctx.WriteString(" (") - ctx.FormatNode(&node.AsColumnNames) + ctx.FormatNode(&node.Defs) ctx.WriteByte(')') } ctx.WriteString(" AS ") diff --git a/pkg/sql/sem/tree/pretty.go b/pkg/sql/sem/tree/pretty.go index 0d8120cfd86d..0a967c573cc9 100644 --- a/pkg/sql/sem/tree/pretty.go +++ b/pkg/sql/sem/tree/pretty.go @@ -1110,9 +1110,9 @@ func (node *CreateTable) doc(p *PrettyCfg) pretty.Doc { title = pretty.ConcatSpace(title, p.Doc(&node.Table)) if node.As() { - if len(node.AsColumnNames) > 0 { + if len(node.Defs) > 0 { title = pretty.ConcatSpace(title, - p.bracket("(", p.Doc(&node.AsColumnNames), ")")) + p.bracket("(", p.Doc(&node.Defs), ")")) } title = pretty.ConcatSpace(title, pretty.Keyword("AS")) } else { @@ -1609,7 +1609,11 @@ func (node *ColumnTableDef) docRow(p *PrettyCfg) pretty.TableRow { clauses := make([]pretty.Doc, 0, 7) // Column type. - clauses = append(clauses, pretty.Text(node.columnTypeString())) + // ColumnTableDef node type will not be specified if it represents a CREATE + // TABLE ... AS query. + if node.Type != nil { + clauses = append(clauses, pretty.Text(node.columnTypeString())) + } // Compute expression (for computed columns). if node.IsComputed() { @@ -1690,10 +1694,26 @@ func (node *ColumnTableDef) docRow(p *PrettyCfg) pretty.TableRow { clauses = append(clauses, p.maybePrependConstraintName(&node.References.ConstraintName, fk)) } - return pretty.TableRow{ - Label: node.Name.String(), - Doc: pretty.Group(pretty.Stack(clauses...)), + // Prevents an additional space from being appended at the end of every column + // name in the case of CREATE TABLE ... AS query. The additional space is + // being caused due to the absence of column type qualifiers in CTAS queries. + // + // TODO(adityamaru): Consult someone with more knowledge about the pretty + // printer architecture to find a cleaner solution. + var tblRow pretty.TableRow + if node.Type == nil { + tblRow = pretty.TableRow{ + Label: node.Name.String(), + Doc: pretty.Stack(clauses...), + } + } else { + tblRow = pretty.TableRow{ + Label: node.Name.String(), + Doc: pretty.Group(pretty.Stack(clauses...)), + } } + + return tblRow } func (node *CheckConstraintTableDef) doc(p *PrettyCfg) pretty.Doc {