Skip to content

Commit

Permalink
pgwire: Add support for cursors enclosed in quotes
Browse files Browse the repository at this point in the history
In CockroachDB and Postgres, it is possible to declare
cursors with invalid identifiers enclosed within double
quotes, for e.g. "1-2-3". However, when attempting to
look up the cursor during pgwire DESCRIBE step, it is
necessary to enclose the cursor name within double quotes
while looking up the cursor name within the CRDB code.
We don't currently do this, which causes the cursor look
up to fail during the DESCRIBE step.
In this PR, the cursor lookup step has been updated to
allow looking up a cursor name created by enclosing it in
double quotes.

Resolves #84261

Release note (bug fix): The pgwire DESCRIBE step no longer
fails with an error while attempting to look up cursors
declared by enclosing the cursor name within double quotes.
  • Loading branch information
rimadeodhar committed Aug 12, 2022
1 parent 94d2b25 commit 8fb34db
Show file tree
Hide file tree
Showing 7 changed files with 105 additions and 37 deletions.
10 changes: 6 additions & 4 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -328,7 +328,7 @@ func (ex *connExecutor) execBind(
return retErr(pgerror.Newf(
pgcode.DuplicateCursor, "portal %q already exists", portalName))
}
if cursor := ex.getCursorAccessor().getCursor(portalName); cursor != nil {
if cursor := ex.getCursorAccessor().getCursor(tree.Name(portalName)); cursor != nil {
return retErr(pgerror.Newf(
pgcode.DuplicateCursor, "portal %q already exists as cursor", portalName))
}
Expand Down Expand Up @@ -484,7 +484,7 @@ func (ex *connExecutor) addPortal(
if _, ok := ex.extraTxnState.prepStmtsNamespace.portals[portalName]; ok {
panic(errors.AssertionFailedf("portal already exists: %q", portalName))
}
if cursor := ex.getCursorAccessor().getCursor(portalName); cursor != nil {
if cursor := ex.getCursorAccessor().getCursor(tree.Name(portalName)); cursor != nil {
panic(errors.AssertionFailedf("portal already exists as cursor: %q", portalName))
}

Expand Down Expand Up @@ -563,7 +563,7 @@ func (ex *connExecutor) execDescribe(

switch descCmd.Type {
case pgwirebase.PrepareStatement:
ps, ok := ex.extraTxnState.prepStmtsNamespace.prepStmts[descCmd.Name]
ps, ok := ex.extraTxnState.prepStmtsNamespace.prepStmts[string(descCmd.Name)]
if !ok {
return retErr(pgerror.Newf(
pgcode.InvalidSQLStatementName,
Expand Down Expand Up @@ -593,7 +593,9 @@ func (ex *connExecutor) execDescribe(
res.SetPrepStmtOutput(ctx, ps.Columns)
}
case pgwirebase.PreparePortal:
portal, ok := ex.extraTxnState.prepStmtsNamespace.portals[descCmd.Name]
// TODO(rimadeodhar): prepStmtsNamespace should also be updated to use tree.Name instead of string
// for indexing internal maps.
portal, ok := ex.extraTxnState.prepStmtsNamespace.portals[string(descCmd.Name)]
if !ok {
// Check SQL-level cursors.
cursor := ex.getCursorAccessor().getCursor(descCmd.Name)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_io.go
Original file line number Diff line number Diff line change
Expand Up @@ -218,7 +218,7 @@ var _ Command = PrepareStmt{}
// DescribeStmt is the Command for producing info about a prepared statement or
// portal.
type DescribeStmt struct {
Name string
Name tree.Name
Type pgwirebase.PrepareType
}

Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -3771,12 +3771,12 @@ https://www.postgresql.org/docs/14/view-pg-cursors.html`,
return err
}
if err := addRow(
tree.NewDString(name), /* name */
tree.NewDString(c.statement), /* statement */
tree.DBoolFalse, /* is_holdable */
tree.DBoolFalse, /* is_binary */
tree.DBoolFalse, /* is_scrollable */
tz, /* creation_date */
tree.NewDString(name.String()), /* name */
tree.NewDString(c.statement), /* statement */
tree.DBoolFalse, /* is_holdable */
tree.DBoolFalse, /* is_binary */
tree.DBoolFalse, /* is_scrollable */
tz, /* creation_date */
); err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/conn.go
Original file line number Diff line number Diff line change
Expand Up @@ -1000,7 +1000,7 @@ func (c *conn) handleDescribe(ctx context.Context, buf *pgwirebase.ReadBuffer) e
return c.stmtBuf.Push(
ctx,
sql.DescribeStmt{
Name: name,
Name: tree.Name(name),
Type: typ,
})
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -694,7 +694,7 @@ func expectDescribeStmt(
t.Fatalf("expected command DescribeStmt, got: %T (%+v)", cmd, cmd)
}

if desc.Name != expName {
if desc.Name.String() != expName {
t.Fatalf("expected name %s, got %s", expName, desc.Name)
}

Expand Down
68 changes: 68 additions & 0 deletions pkg/sql/pgwire/testdata/pgtest/portals
Original file line number Diff line number Diff line change
Expand Up @@ -1486,3 +1486,71 @@ ReadyForQuery
{"Type":"ParseComplete"}
{"Type":"ErrorResponse","Code":"08P01","Message":"invalid DESCRIBE message subtype 0"}
{"Type":"ReadyForQuery","TxStatus":"I"}

# Check declaring cursor with a name enclosed in double quotes
send
Query {"String": "BEGIN"}
----

until
ReadyForQuery
----
{"Type":"CommandComplete","CommandTag":"BEGIN"}
{"Type":"ReadyForQuery","TxStatus":"T"}

send
Parse {"Query": "DECLARE \"1-2-3\" CURSOR FOR SELECT generate_series(1, 10) AS bar"}
Bind
Describe {"ObjectType": "P", "Name": ""}
Execute
Sync
----

until
ReadyForQuery
----
{"Type":"ParseComplete"}
{"Type":"BindComplete"}
{"Type":"NoData"}
{"Type":"CommandComplete","CommandTag":"DECLARE CURSOR"}
{"Type":"ReadyForQuery","TxStatus":"T"}

send
Describe {"ObjectType": "P", "Name": "1-2-3"}
Sync
----

until
ReadyForQuery
----
{"Type":"RowDescription","Fields":[{"Name":"bar","TableOID":0,"TableAttributeNumber":0,"DataTypeOID":20,"DataTypeSize":8,"TypeModifier":-1,"Format":0}]}
{"Type":"ReadyForQuery","TxStatus":"T"}

send
Parse {"Query": "FETCH 2 \"1-2-3\""}
Bind
Describe {"ObjectType": "P", "Name": ""}
Execute
Sync
----

until
ReadyForQuery
----
{"Type":"ParseComplete"}
{"Type":"BindComplete"}
{"Type":"RowDescription","Fields":[{"Name":"bar","TableOID":0,"TableAttributeNumber":0,"DataTypeOID":20,"DataTypeSize":8,"TypeModifier":-1,"Format":0}]}
{"Type":"DataRow","Values":[{"text":"1"}]}
{"Type":"DataRow","Values":[{"text":"2"}]}
{"Type":"CommandComplete","CommandTag":"FETCH 2"}
{"Type":"ReadyForQuery","TxStatus":"T"}

send
Query {"String": "ROLLBACK"}
----

until
ReadyForQuery
----
{"Type":"CommandComplete","CommandTag":"ROLLBACK"}
{"Type":"ReadyForQuery","TxStatus":"I"}
46 changes: 22 additions & 24 deletions pkg/sql/sql_cursor.go
Original file line number Diff line number Diff line change
Expand Up @@ -49,13 +49,12 @@ func (p *planner) DeclareCursor(ctx context.Context, s *tree.DeclareCursor) (pla
}

ie := p.ExecCfg().InternalExecutorFactory.NewInternalExecutor(p.SessionData())
cursorName := s.Name.String()
if cursor := p.sqlCursors.getCursor(cursorName); cursor != nil {
return nil, pgerror.Newf(pgcode.DuplicateCursor, "cursor %q already exists", cursorName)
if cursor := p.sqlCursors.getCursor(s.Name); cursor != nil {
return nil, pgerror.Newf(pgcode.DuplicateCursor, "cursor %q already exists", s.Name)
}

if p.extendedEvalCtx.PreparedStatementState.HasPortal(cursorName) {
return nil, pgerror.Newf(pgcode.DuplicateCursor, "cursor %q already exists as portal", cursorName)
if p.extendedEvalCtx.PreparedStatementState.HasPortal(s.Name.String()) {
return nil, pgerror.Newf(pgcode.DuplicateCursor, "cursor %q already exists as portal", s.Name)
}

// Try to plan the cursor query to make sure that it's valid.
Expand Down Expand Up @@ -100,7 +99,7 @@ func (p *planner) DeclareCursor(ctx context.Context, s *tree.DeclareCursor) (pla
statement: statement,
created: timeutil.Now(),
}
if err := p.sqlCursors.addCursor(cursorName, cursor); err != nil {
if err := p.sqlCursors.addCursor(s.Name, cursor); err != nil {
// This case shouldn't happen because cursor names are scoped to a session,
// and sessions can't have more than one statement running at once. But
// let's be diligent and clean up if it somehow does happen anyway.
Expand All @@ -119,11 +118,10 @@ var errBackwardScan = pgerror.Newf(pgcode.ObjectNotInPrerequisiteState, "cursor
func (p *planner) FetchCursor(
_ context.Context, s *tree.CursorStmt, isMove bool,
) (planNode, error) {
cursorName := s.Name.String()
cursor := p.sqlCursors.getCursor(cursorName)
cursor := p.sqlCursors.getCursor(s.Name)
if cursor == nil {
return nil, pgerror.Newf(
pgcode.InvalidCursorName, "cursor %q does not exist", cursorName,
pgcode.InvalidCursorName, "cursor %q does not exist", s.Name,
)
}
if s.Count < 0 || s.FetchType == tree.FetchBackwardAll {
Expand Down Expand Up @@ -243,7 +241,7 @@ func (p *planner) CloseCursor(ctx context.Context, n *tree.CloseCursor) (planNod
return &delayedNode{
name: n.String(),
constructor: func(ctx context.Context, p *planner) (planNode, error) {
return newZeroNode(nil /* columns */), p.sqlCursors.closeCursor(n.Name.String())
return newZeroNode(nil /* columns */), p.sqlCursors.closeCursor(n.Name)
},
}, nil
}
Expand Down Expand Up @@ -276,20 +274,20 @@ type sqlCursors interface {
closeAll()
// closeCursor closes the named cursor, returning an error if that cursor
// didn't exist in the set.
closeCursor(string) error
closeCursor(tree.Name) error
// getCursor returns the named cursor, returning nil if that cursor
// didn't exist in the set.
getCursor(string) *sqlCursor
getCursor(tree.Name) *sqlCursor
// addCursor adds a new cursor with the given name to the set, returning an
// error if the cursor already existed in the set.
addCursor(string, *sqlCursor) error
addCursor(tree.Name, *sqlCursor) error
// list returns all open cursors in the set.
list() map[string]*sqlCursor
list() map[tree.Name]*sqlCursor
}

// cursorMap is a sqlCursors that's backed by an actual map.
type cursorMap struct {
cursors map[string]*sqlCursor
cursors map[tree.Name]*sqlCursor
}

func (c *cursorMap) closeAll() {
Expand All @@ -299,7 +297,7 @@ func (c *cursorMap) closeAll() {
c.cursors = nil
}

func (c *cursorMap) closeCursor(s string) error {
func (c *cursorMap) closeCursor(s tree.Name) error {
cursor, ok := c.cursors[s]
if !ok {
return pgerror.Newf(pgcode.InvalidCursorName, "cursor %q does not exist", s)
Expand All @@ -309,13 +307,13 @@ func (c *cursorMap) closeCursor(s string) error {
return err
}

func (c *cursorMap) getCursor(s string) *sqlCursor {
func (c *cursorMap) getCursor(s tree.Name) *sqlCursor {
return c.cursors[s]
}

func (c *cursorMap) addCursor(s string, cursor *sqlCursor) error {
func (c *cursorMap) addCursor(s tree.Name, cursor *sqlCursor) error {
if c.cursors == nil {
c.cursors = make(map[string]*sqlCursor)
c.cursors = make(map[tree.Name]*sqlCursor)
}
if _, ok := c.cursors[s]; ok {
return pgerror.Newf(pgcode.DuplicateCursor, "cursor %q already exists", s)
Expand All @@ -324,7 +322,7 @@ func (c *cursorMap) addCursor(s string, cursor *sqlCursor) error {
return nil
}

func (c *cursorMap) list() map[string]*sqlCursor {
func (c *cursorMap) list() map[tree.Name]*sqlCursor {
return c.cursors
}

Expand All @@ -338,19 +336,19 @@ func (c connExCursorAccessor) closeAll() {
c.ex.extraTxnState.sqlCursors.closeAll()
}

func (c connExCursorAccessor) closeCursor(s string) error {
func (c connExCursorAccessor) closeCursor(s tree.Name) error {
return c.ex.extraTxnState.sqlCursors.closeCursor(s)
}

func (c connExCursorAccessor) getCursor(s string) *sqlCursor {
func (c connExCursorAccessor) getCursor(s tree.Name) *sqlCursor {
return c.ex.extraTxnState.sqlCursors.getCursor(s)
}

func (c connExCursorAccessor) addCursor(s string, cursor *sqlCursor) error {
func (c connExCursorAccessor) addCursor(s tree.Name, cursor *sqlCursor) error {
return c.ex.extraTxnState.sqlCursors.addCursor(s, cursor)
}

func (c connExCursorAccessor) list() map[string]*sqlCursor {
func (c connExCursorAccessor) list() map[tree.Name]*sqlCursor {
return c.ex.extraTxnState.sqlCursors.list()
}

Expand Down

0 comments on commit 8fb34db

Please sign in to comment.