Skip to content

Commit

Permalink
Added full Postgres AST to Vitess AST conversion
Browse files Browse the repository at this point in the history
  • Loading branch information
Hydrocharged committed Nov 13, 2023
1 parent b8bbaad commit 4eda5d4
Show file tree
Hide file tree
Showing 148 changed files with 6,646 additions and 130 deletions.
2 changes: 1 addition & 1 deletion format_repo.sh
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@

set -eo pipefail

paths=`find . -maxdepth 1 -mindepth 1 \( -name gen -prune -o -type d -print -o -type f -name '*.go' -print \)`
paths=`find . -maxdepth 1 -mindepth 1 ! -name ".idea" ! -name ".git" ! -name ".github" \( -name gen -prune -o -type d -print -o -type f -name '*.go' -print \)`

goimports -w -local github.com/dolthub/doltgresql $paths

Expand Down
23 changes: 21 additions & 2 deletions postgres/parser/sem/tree/constant.go
Original file line number Diff line number Diff line change
Expand Up @@ -156,12 +156,13 @@ func (expr *NumVal) Kind() constant.Kind {
return expr.value.Kind()
}

// ExactString implements the constant.Value interface.
// ExactString implements the constant.Value interface. Does not include the
// negative sign.
func (expr *NumVal) ExactString() string {
return expr.value.ExactString()
}

// OrigString returns the origString field.
// OrigString returns the origString field. Does not include the negative sign.
func (expr *NumVal) OrigString() string {
return expr.origString
}
Expand Down Expand Up @@ -190,6 +191,19 @@ func (expr *NumVal) Format(ctx *FmtCtx) {
ctx.WriteString(s)
}

// FormattedString returns a string that includes the negative sign.
func (expr *NumVal) FormattedString() string {
s := expr.origString
if s == "" {
s = expr.value.String()
}
if expr.negative {
return "-" + s
} else {
return s
}
}

// canBeInt64 checks if it's possible for the value to become an int64:
// 1 = yes
// 1.0 = yes
Expand Down Expand Up @@ -469,6 +483,11 @@ func (expr *StrVal) Format(ctx *FmtCtx) {
}
}

// WasScannedAsBytes returns true iff the input syntax was using b'...' or x'....'
func (expr *StrVal) WasScannedAsBytes() bool {
return expr.scannedAsBytes
}

var (
// StrValAvailAllParsable is the set of parsable string types.
StrValAvailAllParsable = []*types.T{
Expand Down
5 changes: 2 additions & 3 deletions postgres/parser/sem/tree/create.go
Original file line number Diff line number Diff line change
Expand Up @@ -886,9 +886,8 @@ func (node *ForeignKeyConstraintTableDef) SetName(name Name) {
// CheckConstraintTableDef represents a check constraint within a CREATE
// TABLE statement.
type CheckConstraintTableDef struct {
Name Name
Expr Expr
Hidden bool
Name Name
Expr Expr
}

// SetName implements the ConstraintTableDef interface.
Expand Down
31 changes: 0 additions & 31 deletions postgres/parser/sem/tree/expr.go
Original file line number Diff line number Diff line change
Expand Up @@ -1067,37 +1067,6 @@ func (node *Subquery) Format(ctx *FmtCtx) {
}
}

// TypedDummy is a dummy expression that represents a dummy value with
// a specified type. It can be used in situations where TypedExprs of a
// particular type are required for semantic analysis.
type TypedDummy struct {
Typ *types.T
}

func (node *TypedDummy) String() string {
return AsString(node)
}

// Format implements the NodeFormatter interface.
func (node *TypedDummy) Format(ctx *FmtCtx) {
ctx.WriteString("dummyvalof(")
ctx.FormatTypeReference(node.Typ)
ctx.WriteString(")")
}

// ResolvedType implements the TypedExpr interface.
func (node *TypedDummy) ResolvedType() *types.T {
return node.Typ
}

// TypeCheck implements the Expr interface.
func (node *TypedDummy) TypeCheck(context.Context, *SemaContext, *types.T) (TypedExpr, error) {
return node, nil
}

// Walk implements the Expr interface.
func (node *TypedDummy) Walk(Visitor) Expr { return node }

// BinaryOperator represents a binary operator.
type BinaryOperator int

Expand Down
12 changes: 0 additions & 12 deletions postgres/parser/sem/tree/values.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,15 +50,3 @@ func (node *ValuesClause) Format(ctx *FmtCtx) {
comma = ", "
}
}

// ValuesClauseWithNames is a VALUES clause that has been annotated with column
// names. This is only produced at plan time, never by the parser. It's used to
// pass column names to the VALUES planNode, so it can produce intelligible
// error messages during value type checking.
type ValuesClauseWithNames struct {
ValuesClause

// Names is a list of the column names that each tuple in the ValuesClause
// corresponds to.
Names NameList
}
82 changes: 82 additions & 0 deletions server/ast/aliased_table_expr.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
// Copyright 2023 Dolthub, Inc.
//
// 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.

package ast

import (
"fmt"
"sync/atomic"

vitess "github.com/dolthub/vitess/go/vt/sqlparser"

"github.com/dolthub/doltgresql/postgres/parser/sem/tree"
)

// uniqueAliasCounter is used to create unique aliases. Aliases are required by GMS when some expressions are contained
// within a vitess.AliasedTableExpr. The Postgres AST does not have this restriction, so we must do this for
// compatibility. Callers are free to change the alias if need be, but we'll always set an alias to be safe.
var uniqueAliasCounter atomic.Uint64

// nodeAliasedTableExpr handles *tree.AliasedTableExpr nodes.
func nodeAliasedTableExpr(node *tree.AliasedTableExpr) (*vitess.AliasedTableExpr, error) {
if node.Ordinality {
return nil, fmt.Errorf("ordinality is not yet supported")
}
if node.IndexFlags != nil {
return nil, fmt.Errorf("index flags are not yet supported")
}
var aliasExpr vitess.SimpleTableExpr
switch expr := node.Expr.(type) {
case *tree.TableName:
var err error
aliasExpr, err = nodeTableName(expr)
if err != nil {
return nil, err
}
default:
tableExpr, err := nodeTableExpr(expr)
if err != nil {
return nil, err
}
subquery := &vitess.Subquery{
Select: &vitess.Select{
From: vitess.TableExprs{tableExpr},
},
}
//TODO: make sure that this actually works
if len(node.As.Cols) > 0 {
columns := make([]vitess.ColIdent, len(node.As.Cols))
for i := range node.As.Cols {
columns[i] = vitess.NewColIdent(string(node.As.Cols[i]))
}
subquery.Columns = columns
}
aliasExpr = subquery
}
alias := string(node.As.Alias)
if len(alias) == 0 {
alias = generateUniqueAlias()
}
return &vitess.AliasedTableExpr{
Expr: aliasExpr,
As: vitess.NewTableIdent(alias),
AsOf: nil,
Lateral: node.Lateral,
}, nil
}

// generateUniqueAlias generates a unique alias. This is thread-safe.
func generateUniqueAlias() string {
return fmt.Sprintf("doltgres!|alias|%d", uniqueAliasCounter.Add(1))
}
31 changes: 31 additions & 0 deletions server/ast/alter_database_owner.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2023 Dolthub, Inc.
//
// 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.

package ast

import (
"fmt"

vitess "github.com/dolthub/vitess/go/vt/sqlparser"

"github.com/dolthub/doltgresql/postgres/parser/sem/tree"
)

// nodeAlterDatabaseOwner handles *tree.AlterDatabaseOwner nodes.
func nodeAlterDatabaseOwner(node *tree.AlterDatabaseOwner) (vitess.Statement, error) {
if node == nil {
return nil, nil
}
return nil, fmt.Errorf("ALTER DATABASE OWNER is not yet supported")
}
32 changes: 32 additions & 0 deletions server/ast/alter_index.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
// Copyright 2023 Dolthub, Inc.
//
// 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.

package ast

import (
"fmt"

vitess "github.com/dolthub/vitess/go/vt/sqlparser"

"github.com/dolthub/doltgresql/postgres/parser/sem/tree"
)

// nodeAlterIndex handles *tree.AlterIndex nodes.
func nodeAlterIndex(node *tree.AlterIndex) (vitess.Statement, error) {
if node == nil {
return nil, nil
}
// Only PARTITION alterations are supported by the parser, so there's nothing to convert to yet
return nil, fmt.Errorf("ALTER INDEX is not yet supported")
}
31 changes: 31 additions & 0 deletions server/ast/alter_role.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2023 Dolthub, Inc.
//
// 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.

package ast

import (
"fmt"

vitess "github.com/dolthub/vitess/go/vt/sqlparser"

"github.com/dolthub/doltgresql/postgres/parser/sem/tree"
)

// nodeAlterRole handles *tree.AlterRole nodes.
func nodeAlterRole(node *tree.AlterRole) (vitess.Statement, error) {
if node == nil {
return nil, nil
}
return nil, fmt.Errorf("ALTER ROLE is not yet supported")
}
31 changes: 31 additions & 0 deletions server/ast/alter_schema.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2023 Dolthub, Inc.
//
// 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.

package ast

import (
"fmt"

vitess "github.com/dolthub/vitess/go/vt/sqlparser"

"github.com/dolthub/doltgresql/postgres/parser/sem/tree"
)

// nodeAlterSchema handles *tree.AlterSchema nodes.
func nodeAlterSchema(node *tree.AlterSchema) (vitess.Statement, error) {
if node == nil {
return nil, nil
}
return nil, fmt.Errorf("ALTER SCHEMA is not yet supported")
}
31 changes: 31 additions & 0 deletions server/ast/alter_sequence.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
// Copyright 2023 Dolthub, Inc.
//
// 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.

package ast

import (
"fmt"

vitess "github.com/dolthub/vitess/go/vt/sqlparser"

"github.com/dolthub/doltgresql/postgres/parser/sem/tree"
)

// nodeAlterSequence handles *tree.AlterSequence nodes.
func nodeAlterSequence(node *tree.AlterSequence) (vitess.Statement, error) {
if node == nil {
return nil, nil
}
return nil, fmt.Errorf("ALTER SEQUENCE is not yet supported")
}
Loading

0 comments on commit 4eda5d4

Please sign in to comment.