diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index cb2f279bb216..776a9b6a800b 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -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)) } @@ -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)) } @@ -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, @@ -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) diff --git a/pkg/sql/conn_io.go b/pkg/sql/conn_io.go index bc0fb4443e6f..f0861ed00df9 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/logictest/testdata/logic_test/cursor b/pkg/sql/logictest/testdata/logic_test/cursor index 46c307b91b04..733e9a32fddb 100644 --- a/pkg/sql/logictest/testdata/logic_test/cursor +++ b/pkg/sql/logictest/testdata/logic_test/cursor @@ -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; + diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog index 18f786594687..e4d40d3674d3 100644 --- a/pkg/sql/logictest/testdata/logic_test/pg_catalog +++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog @@ -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; diff --git a/pkg/sql/pg_catalog.go b/pkg/sql/pg_catalog.go index 83bd1b7abcef..af8f5e5eb544 100644 --- a/pkg/sql/pg_catalog.go +++ b/pkg/sql/pg_catalog.go @@ -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 } diff --git a/pkg/sql/pgwire/conn.go b/pkg/sql/pgwire/conn.go index cc7720eaa3c6..b6ba058e6c17 100644 --- a/pkg/sql/pgwire/conn.go +++ b/pkg/sql/pgwire/conn.go @@ -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, }) } diff --git a/pkg/sql/pgwire/conn_test.go b/pkg/sql/pgwire/conn_test.go index b88ddf1712dc..eabe43a9c1d1 100644 --- a/pkg/sql/pgwire/conn_test.go +++ b/pkg/sql/pgwire/conn_test.go @@ -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) @@ -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) @@ -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, diff --git a/pkg/sql/pgwire/testdata/pgtest/portals b/pkg/sql/pgwire/testdata/pgtest/portals index 55c0595da6df..3f0b9a472bc9 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 \"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"} diff --git a/pkg/sql/sql_cursor.go b/pkg/sql/sql_cursor.go index 26b9609e1646..2499898ce00a 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(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. @@ -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() }