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 special identifiers enclosed within double
quotes, for e.g. "1-2-3". Currently, we store the name
as an unescaped string which causes problems during the
pgwire DESCRIBE step for looking up the cursor. We should
be storing using the tree.Name datatype for the cursor name
while storing and looking up cursors. This PR updates the code
to start using tree.Name instead of raw strings for handling
cursor names. This fixes the issue where the pgwire DESCRIBE
step fails while attempting to look up cursors with names
containing special characters.

Resolves #84261

Release note (bug fix): The pgwire DESCRIBE step no longer
fails with an error while attempting to look up cursors
declared with names containing special characters.
  • Loading branch information
rimadeodhar committed Sep 20, 2022
1 parent 9b3b3b6 commit c0aa573
Show file tree
Hide file tree
Showing 9 changed files with 181 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 @@ -337,7 +337,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 @@ -493,7 +493,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 @@ -572,7 +572,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 @@ -602,7 +602,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
42 changes: 42 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/cursor
Original file line number Diff line number Diff line change
Expand Up @@ -557,3 +557,45 @@ CLOSE foo

statement ok
ALTER TABLE a ADD COLUMN c INT

statement ok
COMMIT;

statement ok
BEGIN;

statement ok
DECLARE "a"" b'c" CURSOR FOR SELECT 1;

query I
FETCH 1 "a"" b'c";
----
1

statement ok
CLOSE "a"" b'c";
DECLARE "a b" CURSOR FOR SELECT 2;

query I
FETCH 1 "a b";
----
2

statement ok
CLOSE "a b";
DECLARE "a\b" CURSOR FOR SELECT 3;

query I
FETCH 1 "a\b";
----
3

statement ok
CLOSE "a\b";

query error pq: at or near "b": syntax error
FETCH 1 a b;

statement ok
COMMIT;

28 changes: 28 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -6034,3 +6034,31 @@ SELECT * FROM pg_sequences WHERE sequencename = 'serial'
----
schemaname sequencename sequenceowner data_type start_value min_value max_value increment_by cycle cache_size last_value
public serial root 20 101 1 9223372036854775807 5 false 1 NULL

statement ok
CREATE TABLE t (a INT PRIMARY KEY, b INT);
INSERT INTO t VALUES (1, 2), (2, 3);

statement ok
BEGIN;

statement ok
DECLARE "a"" b'c" CURSOR FOR TABLE t;
DECLARE "a b" CURSOR FOR TABLE t;
DECLARE "a\b" CURSOR FOR TABLE t;

## pg_catalog.pg_cursors

query TTBBB colnames
SELECT name, statement, is_holdable, is_binary, is_scrollable FROM pg_catalog.pg_cursors ORDER BY name;
----
name statement is_holdable is_binary is_scrollable
a b TABLE t false false false
a" b'c TABLE t false false false
a\b TABLE t false false false

statement ok
COMMIT;

statement ok
DROP TABLE t;
12 changes: 6 additions & 6 deletions pkg/sql/pg_catalog.go
Original file line number Diff line number Diff line change
Expand Up @@ -3776,12 +3776,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(string(name)), /* 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 @@ -1011,7 +1011,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
8 changes: 7 additions & 1 deletion pkg/sql/pgwire/conn_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -159,6 +159,9 @@ func TestConn(t *testing.T) {
if err := finishQuery(generateError, conn); err != nil {
t.Fatal(err)
}
expectExecStmt(ctx, t, "DECLARE \"a b\" CURSOR FOR SELECT 10", &rd, conn, queryStringComplete)
expectExecStmt(ctx, t, "FETCH 1 \"a b\"", &rd, conn, queryStringComplete)
expectExecStmt(ctx, t, "CLOSE \"a b\"", &rd, conn, queryStringComplete)
// We got to the COMMIT at the end of the batch.
expectExecStmt(ctx, t, "COMMIT TRANSACTION", &rd, conn, queryStringComplete)
expectSync(ctx, t, &rd)
Expand Down Expand Up @@ -505,6 +508,9 @@ func client(ctx context.Context, serverAddr net.Addr, wg *sync.WaitGroup) error
batch.Queue("BEGIN")
batch.Queue("select 7")
batch.Queue("select 8")
batch.Queue("declare \"a b\" cursor for select 10")
batch.Queue("fetch 1 \"a b\"")
batch.Queue("close \"a b\"")
batch.Queue("COMMIT")

batchResults := conn.SendBatch(ctx, batch)
Expand Down Expand Up @@ -676,7 +682,7 @@ func expectPrepareStmt(
func expectDescribeStmt(
ctx context.Context,
t *testing.T,
expName string,
expName tree.Name,
expType pgwirebase.PrepareType,
rd *sql.StmtBufReader,
c *conn,
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 \"a b\" 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": "a b"}
Sync
----

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

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

until ignore_type_oids ignore_table_oids ignore_data_type_sizes
ReadyForQuery
----
{"Type":"ParseComplete"}
{"Type":"BindComplete"}
{"Type":"RowDescription","Fields":[{"Name":"bar","TableOID":0,"TableAttributeNumber":0,"DataTypeOID":0,"DataTypeSize":0,"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(string(s.Name)) {
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 c0aa573

Please sign in to comment.