From 8fb34db0f5bb35d48ad91d6d8e8946719bad4871 Mon Sep 17 00:00:00 2001 From: rimadeodhar Date: Mon, 25 Jul 2022 13:58:35 -0700 Subject: [PATCH] pgwire: Add support for cursors enclosed in quotes 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 https://github.com/cockroachdb/cockroach/issues/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. --- pkg/sql/conn_executor_prepare.go | 10 ++-- pkg/sql/conn_io.go | 2 +- pkg/sql/pg_catalog.go | 12 ++--- pkg/sql/pgwire/conn.go | 2 +- pkg/sql/pgwire/conn_test.go | 2 +- pkg/sql/pgwire/testdata/pgtest/portals | 68 ++++++++++++++++++++++++++ pkg/sql/sql_cursor.go | 46 +++++++++-------- 7 files changed, 105 insertions(+), 37 deletions(-) diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index eca9dc3aa559..bca91f6fe04a 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -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)) } @@ -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)) } @@ -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, @@ -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) diff --git a/pkg/sql/conn_io.go b/pkg/sql/conn_io.go index e6290165e9f5..7e11384152ff 100644 --- a/pkg/sql/conn_io.go +++ b/pkg/sql/conn_io.go @@ -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 } diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index b45f4be0cbe2..7ba5063605ce 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -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 } diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go index a50a722c9a4b..f16ef81988bd 100644 --- a/pkg/sql/pgwire/conn.go +++ b/pkg/sql/pgwire/conn.go @@ -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, }) } diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index 66b73b4e51c7..cd8c89f74e03 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -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) } diff --git a/pkg/sql/pgwire/testdata/pgtest/portals b/pkg/sql/pgwire/testdata/pgtest/portals index 55c0595da6df..2a640b94dfaf 100644 --- a/pkg/sql/pgwire/testdata/pgtest/portals +++ b/pkg/sql/pgwire/testdata/pgtest/portals @@ -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"} diff --git a/pkg/sql/sql_cursor.go b/pkg/sql/sql_cursor.go index 26b9609e1646..0096af0132d5 100644 --- a/pkg/sql/sql_cursor.go +++ b/pkg/sql/sql_cursor.go @@ -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. @@ -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. @@ -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 { @@ -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 } @@ -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() { @@ -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) @@ -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) @@ -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 } @@ -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() }