Skip to content

Commit

Permalink
server,sql: Make the statement struct generic
Browse files Browse the repository at this point in the history
This commit makes Statement generic so it can be shared between
the sql and plpgsql parsers.

Release note: None
  • Loading branch information
e-mbrown committed May 10, 2023
1 parent 16acd8c commit 0216286
Show file tree
Hide file tree
Showing 44 changed files with 278 additions and 227 deletions.
2 changes: 2 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -1800,6 +1800,7 @@ GO_TARGETS = [
"//pkg/sql/optionalnodeliveness:optionalnodeliveness",
"//pkg/sql/paramparse:paramparse",
"//pkg/sql/parser/fuzz:fuzz",
"//pkg/sql/parser/statements:statements",
"//pkg/sql/parser:parser",
"//pkg/sql/parser:parser_test",
"//pkg/sql/pgwire/hba:hba",
Expand Down Expand Up @@ -3047,6 +3048,7 @@ GET_X_DATA_TARGETS = [
"//pkg/sql/paramparse:get_x_data",
"//pkg/sql/parser:get_x_data",
"//pkg/sql/parser/fuzz:get_x_data",
"//pkg/sql/parser/statements:get_x_data",
"//pkg/sql/pgwire:get_x_data",
"//pkg/sql/pgwire/hba:get_x_data",
"//pkg/sql/pgwire/identmap:get_x_data",
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -162,6 +162,7 @@ go_library(
"//pkg/sql/execinfrapb",
"//pkg/sql/lexbase",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/protoreflect",
"//pkg/sql/row",
Expand Down
3 changes: 2 additions & 1 deletion pkg/cli/sqlfmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"os"

"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
Expand Down Expand Up @@ -43,7 +44,7 @@ func runSQLFmt(cmd *cobra.Command, args []string) error {
return errors.Errorf("tab width must be > 0: %d", sqlfmtCtx.tabWidth)
}

var sl parser.Statements
var sl statements.Statements
if len(sqlfmtCtx.execStmts) != 0 {
for _, exec := range sqlfmtCtx.execStmts {
stmts, err := parser.Parse(exec)
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/reduce/reduce/reducesql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_library(
deps = [
"//pkg/cmd/reduce/reduce",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/sem/builtins",
"//pkg/sql/sem/tree",
],
Expand Down
3 changes: 2 additions & 1 deletion pkg/cmd/reduce/reduce/reducesql/reducesql.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/cmd/reduce/reduce"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
// Import builtins.
_ "github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -399,7 +400,7 @@ func Pretty(s string) (string, error) {
return joinASTs(collectASTs(stmts)), nil
}

func collectASTs(stmts parser.Statements) []tree.NodeFormatter {
func collectASTs(stmts statements.Statements) []tree.NodeFormatter {
asts := make([]tree.NodeFormatter, len(stmts))
for i, stmt := range stmts {
asts[i] = stmt.AST
Expand Down
1 change: 1 addition & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -225,6 +225,7 @@ go_library(
"//pkg/sql/lexbase",
"//pkg/sql/optionalnodeliveness",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/pgwire",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
Expand Down
7 changes: 4 additions & 3 deletions pkg/server/api_v2_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/isql"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
Expand Down Expand Up @@ -204,9 +205,9 @@ func (a *apiV2Server) execSQL(w http.ResponseWriter, r *http.Request) {
ApplicationName string `json:"application_name"`
Execute bool `json:"execute"`
Statements []struct {
SQL string `json:"sql"`
stmt parser.Statement `json:"-"`
Arguments []interface{} `json:"arguments,omitempty"`
SQL string `json:"sql"`
stmt statements.Statement[tree.Statement] `json:"-"`
Arguments []interface{} `json:"arguments,omitempty"`
} `json:"statements"`
}
// Type for the result.
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,7 @@ go_library(
"//pkg/sql/optionalnodeliveness",
"//pkg/sql/paramparse",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/pgwire/pgnotice",
Expand Down Expand Up @@ -786,6 +787,7 @@ go_test(
"//pkg/sql/mutations",
"//pkg/sql/opt/exec/explain",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/pgwire/pgwirebase",
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/idxrecommendations"
"github.com/cockroachdb/cockroach/pkg/sql/idxusage"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirecancel"
Expand Down Expand Up @@ -2894,7 +2895,7 @@ func (ex *connExecutor) execCopyOut(
return nil, nil
}

func (ex *connExecutor) setCopyLoggingFields(stmt parser.Statement) {
func (ex *connExecutor) setCopyLoggingFields(stmt statements.Statement[tree.Statement]) {
// These fields need to be set for logging purposes.
ex.planner.stmt = Statement{
Statement: stmt,
Expand Down
9 changes: 5 additions & 4 deletions pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/exec/explain"
"github.com/cockroachdb/cockroach/pkg/sql/paramparse"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/physicalplan"
Expand Down Expand Up @@ -94,7 +95,7 @@ const numTxnRetryErrors = 3
// then the statement cannot have any placeholder.
func (ex *connExecutor) execStmt(
ctx context.Context,
parserStmt parser.Statement,
parserStmt statements.Statement[tree.Statement],
portal *PreparedPortal,
pinfo *tree.PlaceholderInfo,
res RestrictedCommandResult,
Expand Down Expand Up @@ -274,7 +275,7 @@ func (ex *connExecutor) execPortal(
// The returned event can be nil if no state transition is required.
func (ex *connExecutor) execStmtInOpenState(
ctx context.Context,
parserStmt parser.Statement,
parserStmt statements.Statement[tree.Statement],
portal *PreparedPortal,
pinfo *tree.PlaceholderInfo,
res RestrictedCommandResult,
Expand Down Expand Up @@ -815,7 +816,7 @@ func (ex *connExecutor) execStmtInOpenState(
}
}
prepStmt := makeStatement(
parser.Statement{
statements.Statement[tree.Statement]{
// We need the SQL string just for the part that comes after
// "PREPARE ... AS",
// TODO(radu): it would be nice if the parser would figure out this
Expand Down Expand Up @@ -2652,7 +2653,7 @@ func (ex *connExecutor) enableTracing(modes []string) error {

// addActiveQuery adds a running query to the list of running queries.
func (ex *connExecutor) addActiveQuery(
stmt parser.Statement,
stmt statements.Statement[tree.Statement],
placeholders *tree.PlaceholderInfo,
queryID clusterunique.ID,
cancelQuery context.CancelFunc,
Expand Down
4 changes: 3 additions & 1 deletion pkg/sql/conn_executor_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,8 +30,10 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/querycache"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/testutils"
Expand Down Expand Up @@ -236,7 +238,7 @@ func TestPortalsDestroyedOnTxnFinish(t *testing.T) {
}
}

func mustParseOne(s string) parser.Statement {
func mustParseOne(s string) statements.Statement[tree.Statement] {
stmts, err := parser.Parse(s)
if err != nil {
log.Fatalf(context.Background(), "%v", err)
Expand Down
10 changes: 5 additions & 5 deletions pkg/sql/conn_io.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/col/coldata"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
Expand Down Expand Up @@ -127,7 +127,7 @@ type ExecStmt struct {
// Information returned from parsing: AST, SQL, NumPlaceholders.
// Note that AST can be nil, in which case executing it should produce an
// "empty query response" message.
parser.Statement
statements.Statement[tree.Statement]

// TimeReceived is the time at which the exec message was received
// from the client. Used to compute the service latency.
Expand Down Expand Up @@ -198,7 +198,7 @@ type PrepareStmt struct {
// Information returned from parsing: AST, SQL, NumPlaceholders.
// Note that AST can be nil, in which case executing it should produce an
// "empty query response" message.
parser.Statement
statements.Statement[tree.Statement]

TypeHints tree.PlaceholderTypes
// RawTypeHints is the representation of type hints exactly as specified by
Expand Down Expand Up @@ -330,7 +330,7 @@ var _ Command = Flush{}

// CopyIn is the command for execution of the Copy-in pgwire subprotocol.
type CopyIn struct {
ParsedStmt parser.Statement
ParsedStmt statements.Statement[tree.Statement]
Stmt *tree.CopyFrom
// Conn is the network connection. Execution of the CopyFrom statement takes
// control of the connection.
Expand Down Expand Up @@ -362,7 +362,7 @@ var _ Command = CopyIn{}

// CopyOut is the command for execution of the Copy-out pgwire subprotocol.
type CopyOut struct {
ParsedStmt parser.Statement
ParsedStmt statements.Statement[tree.Statement]
Stmt *tree.CopyTo
// TimeReceived is the time at which the message was received
// from the client. Used to compute the service latency.
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/distsql_plan_changefeed.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/opt/cat"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catid"
Expand Down Expand Up @@ -82,7 +82,7 @@ func PlanCDCExpression(
opt.apply(&cfg)
}

p.stmt = makeStatement(parser.Statement{
p.stmt = makeStatement(statements.Statement[tree.Statement]{
AST: cdcExpr,
SQL: tree.AsString(cdcExpr),
}, clusterunique.ID{} /* queryID */)
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgnotice"
Expand Down Expand Up @@ -2073,7 +2074,7 @@ type queryMeta struct {
start time.Time

// The SQL statement being executed.
stmt parser.Statement
stmt statements.Statement[tree.Statement]

// The placeholders that the query was executed with if any.
placeholders *tree.PlaceholderInfo
Expand Down
19 changes: 8 additions & 11 deletions pkg/sql/lexbase/sql-gen.sh
Original file line number Diff line number Diff line change
Expand Up @@ -6,32 +6,29 @@

set -euo pipefail

if [ "$6" != plpgsql ]; then
SYMUNION="sqlSymUnion"
GENYACC=sql-gen.y
else
SYMUNION="plpgsqlSymUnion"
GENYACC=plpgsql-gen.y
fi;
LANG=$2
SYMUNION="${LANG}"'SymUnion'
GENYACC=$LANG-gen.y


awk -v regex="$SYMUNION" '/func.*'"$SYMUNION"'/ {print $(NF - 1)}' $1 | \
sed -e 's/[]\/$*.^|[]/\\&/g' | \
sed -e "s/^/s_(type|token) <(/" | \
awk '{print $0")>_\\1 <union> /* <\\2> */_"}' > types_regex.tmp

sed -E -f types_regex.tmp < $1 | \
if [ "$6" != plpgsql ]; then \
awk -f $2 | \
if [ $LANG != plpgsql ]; then \
awk -f $3 | \
sed -Ee 's,//.*$$,,g;s,/[*]([^*]|[*][^/])*[*]/, ,g;s/ +$$//g' > $GENYACC
else
sed -Ee 's,//.*$$,,g;s,/[*]([^*]|[*][^/])*[*]/, ,g;s/ +$$//g' > $GENYACC
fi;

rm types_regex.tmp

ret=$($4 -p $6 -o $3 $GENYACC); \
ret=$($5 -p $LANG -o $4 $GENYACC); \
if expr "$ret" : ".*conflicts" >/dev/null; then \
echo "$ret"; exit 1; \
fi;
rm $GENYACC
$5 -w $3
$6 -w $4
1 change: 1 addition & 0 deletions pkg/sql/opt/bench/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ go_test(
"//pkg/sql/opt/testutils/testcat",
"//pkg/sql/opt/xform",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/sem/eval",
"//pkg/sql/sem/tree",
"//pkg/sql/sem/volatility",
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/bench/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,6 +33,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/testutils/testcat"
"github.com/cockroachdb/cockroach/pkg/sql/opt/xform"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/sem/eval"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sem/volatility"
Expand Down Expand Up @@ -849,7 +850,7 @@ func (h *harness) runSimple(tb testing.TB, query benchQuery, phase Phase) {
root,
&h.evalCtx,
true, /* allowAutoCommit */
stmt.IsANSIDML(),
statements.IsANSIDML(stmt.AST),
)
if _, err = eb.Build(); err != nil {
tb.Fatalf("%v", err)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/optbuilder/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ go_library(
"//pkg/sql/opt/props",
"//pkg/sql/opt/props/physical",
"//pkg/sql/parser",
"//pkg/sql/parser/statements",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/privilege",
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/optbuilder/select.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/opt/props"
"github.com/cockroachdb/cockroach/pkg/sql/parser"
"github.com/cockroachdb/cockroach/pkg/sql/parser/statements"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/privilege"
Expand Down Expand Up @@ -590,7 +591,7 @@ func (b *Builder) buildScan(

// Scanning tables in databases that don't use the SURVIVE ZONE FAILURE option
// is disallowed when EnforceHomeRegion is true.
if b.evalCtx.SessionData().EnforceHomeRegion && parser.IsANSIDML(b.stmt) {
if b.evalCtx.SessionData().EnforceHomeRegion && statements.IsANSIDML(b.stmt) {
errorOnInvalidMultiregionDB(b.ctx, b.evalCtx, tabMeta)
// Populate the remote regions touched by the multiregion database used in
// this query. If a query dynamically errors out as having no home region,
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/parser/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_library(
"//pkg/geo/geopb", # keep
"//pkg/security/username", # keep
"//pkg/sql/lexbase",
"//pkg/sql/parser/statements",
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/privilege", # keep
Expand Down Expand Up @@ -98,10 +99,10 @@ GO_ABS_PATH=`cd $$GO_REL_PATH && pwd`
export PATH=$$GO_ABS_PATH:$$PATH
export HOME=$(GENDIR)
export GOPATH=/nonexist-gopath
$(location :sql-gen) $(location sql.y) $(location replace_help_rules.awk) \
export GOROOT=
$(location :sql-gen) $(location sql.y) sql $(location replace_help_rules.awk) \
$(location sql.go) $(location @org_golang_x_tools//cmd/goyacc) \
$(location @com_github_cockroachdb_gostdlib//x/tools/cmd/goimports) \
sql
""",
exec_tools = [
":sql-gen",
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/parser/lexer.go
Original file line number Diff line number Diff line change
Expand Up @@ -397,8 +397,7 @@ func (l *lexer) Error(e string) {
l.populateErrorDetails()
}

// PopulateErrorDetails was created so plpgsql lexer and sql can share a
// single function
// PopulateErrorDetails properly wraps the "last error" field in the lexer.
func PopulateErrorDetails(
tokID int32, lastTokStr string, lastTokPos int32, lastErr error, lIn string,
) error {
Expand Down
Loading

0 comments on commit 0216286

Please sign in to comment.