diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index e65034dea1ed..5aca568495d0 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -428,7 +428,7 @@ func (ex *connExecutor) execStmtInOpenState( return makeErrEvent(err) } var err error - pinfo, err = fillInPlaceholders(ctx, ps, name, e.Params, ex.sessionData().SearchPath) + pinfo, err = ex.planner.fillInPlaceholders(ctx, ps, name, e.Params) if err != nil { return makeErrEvent(err) } diff --git a/pkg/sql/conn_executor_prepare.go b/pkg/sql/conn_executor_prepare.go index 57c26c86b6a7..9abcecba59a7 100644 --- a/pkg/sql/conn_executor_prepare.go +++ b/pkg/sql/conn_executor_prepare.go @@ -372,33 +372,63 @@ func (ex *connExecutor) execBind( "expected %d arguments, got %d", numQArgs, len(bindCmd.Args))) } - for i, arg := range bindCmd.Args { - k := tree.PlaceholderIdx(i) - t := ps.InferredTypes[i] - if arg == nil { - // nil indicates a NULL argument value. - qargs[k] = tree.DNull - } else { - typ, ok := types.OidToType[t] - if !ok { - var err error - typ, err = ex.planner.ResolveTypeByOID(ctx, t) + // We need to make sure type resolution happens within a transaction. + // Otherwise, for user-defined types we won't take the correct leases and + // will get back out of date type information. + // This code path is only used by the wire-level Bind command. The + // SQL EXECUTE command (which also needs to bind and resolve types) is + // handled separately in conn_executor_exec. + resolve := func(ctx context.Context, txn *kv.Txn) (err error) { + ex.statsCollector.Reset(ex.applicationStats, ex.phaseTimes) + p := &ex.planner + ex.resetPlanner(ctx, p, txn, ex.server.cfg.Clock.PhysicalTime() /* stmtTS */) + + for i, arg := range bindCmd.Args { + k := tree.PlaceholderIdx(i) + t := ps.InferredTypes[i] + if arg == nil { + // nil indicates a NULL argument value. + qargs[k] = tree.DNull + } else { + typ, ok := types.OidToType[t] + if !ok { + var err error + typ, err = ex.planner.ResolveTypeByOID(ctx, t) + if err != nil { + return err + } + } + d, err := pgwirebase.DecodeDatum( + ex.planner.EvalContext(), + typ, + qArgFormatCodes[i], + arg, + ) if err != nil { - return nil, err + return pgerror.Wrapf(err, pgcode.ProtocolViolation, "error in argument for %s", k) } + qargs[k] = d } - d, err := pgwirebase.DecodeDatum( - ex.planner.EvalContext(), - typ, - qArgFormatCodes[i], - arg, - ) - if err != nil { - return retErr(pgerror.Wrapf(err, pgcode.ProtocolViolation, - "error in argument for %s", k)) - } - qargs[k] = d } + return nil + } + + if txn := ex.state.mu.txn; txn != nil && txn.IsOpen() { + // Use the existing transaction. + if err := resolve(ctx, txn); err != nil { + return retErr(err) + } + } else { + // Use a new transaction. This will handle retriable errors here rather + // than bubbling them up to the connExecutor state machine. + if err := ex.server.cfg.DB.Txn(ctx, resolve); err != nil { + return retErr(err) + } + // Bind with an implicit transaction will end up creating + // a new transaction. Once this transaction is complete, + // we can safely release the leases, otherwise we will + // incorrectly hold leases for later operations. + ex.extraTxnState.descCollection.ReleaseAll(ctx) } } diff --git a/pkg/sql/execute.go b/pkg/sql/execute.go index 74c1535578fc..8e3f949218e5 100644 --- a/pkg/sql/execute.go +++ b/pkg/sql/execute.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" - "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/errors" ) @@ -25,12 +24,8 @@ import ( // a prepared statement returning // the referenced prepared statement and correctly updated placeholder info. // See https://www.postgresql.org/docs/current/static/sql-execute.html for details. -func fillInPlaceholders( - ctx context.Context, - ps *PreparedStatement, - name string, - params tree.Exprs, - searchPath sessiondata.SearchPath, +func (p *planner) fillInPlaceholders( + ctx context.Context, ps *PreparedStatement, name string, params tree.Exprs, ) (*tree.PlaceholderInfo, error) { if len(ps.Types) != len(params) { return nil, pgerror.Newf(pgcode.Syntax, @@ -47,6 +42,16 @@ func fillInPlaceholders( if !ok { return nil, errors.AssertionFailedf("no type for placeholder %s", idx) } + + // For user-defined types, we need to resolve the type to make sure we get + // the latest changes to the type. + if typ.UserDefined() { + var err error + typ, err = p.ResolveTypeByOID(ctx, typ.Oid()) + if err != nil { + return nil, err + } + } typedExpr, err := schemaexpr.SanitizeVarFreeExpr( ctx, e, typ, "EXECUTE parameter" /* context */, &semaCtx, tree.VolatilityVolatile, ) diff --git a/pkg/sql/logictest/testdata/logic_test/enums b/pkg/sql/logictest/testdata/logic_test/enums index 3f35a32e1af6..ed47beb2333e 100644 --- a/pkg/sql/logictest/testdata/logic_test/enums +++ b/pkg/sql/logictest/testdata/logic_test/enums @@ -1545,3 +1545,27 @@ ORDER BY a.id, b.id {a,b} {a} false false false false {a,b} {b} false true true false {a,b} {a,b} true false true true + +# Make sure that adding a new enum value works well with PREPARE/EXECUTE. +subtest regression_70378 + +statement ok +CREATE TYPE enum_70378 AS ENUM ('a', 'b'); +CREATE TABLE enum_table (a enum_70378); +PREPARE q AS INSERT INTO enum_table VALUES($1); +EXECUTE q('a') + +query T +SELECT * FROM enum_table +---- +a + +statement ok +ALTER TYPE enum_70378 ADD VALUE 'c'; +EXECUTE q('c') + +query T rowsort +SELECT * FROM enum_table +---- +a +c diff --git a/pkg/sql/pgwire/testdata/pgtest/bind_and_resolve b/pkg/sql/pgwire/testdata/pgtest/bind_and_resolve index 5afa65ba63e7..0414556584f1 100644 --- a/pkg/sql/pgwire/testdata/pgtest/bind_and_resolve +++ b/pkg/sql/pgwire/testdata/pgtest/bind_and_resolve @@ -38,15 +38,14 @@ ReadyForQuery # planner transaction but never set it. This was pretty much the only # way you could do such a thing. -send +# This is crdb_only because Postgres does not support AS OF SYSTEM TIME. +send crdb_only Query {"String": "BEGIN AS OF SYSTEM TIME '1s'"} Sync ---- - -# TODO(ajwerner): Why are there two ReadyForQuery? - -until +# There are two ReadyForQuerys because a simple query was followed by Sync. +until crdb_only ErrorResponse ReadyForQuery ReadyForQuery @@ -55,16 +54,93 @@ ReadyForQuery {"Type":"ReadyForQuery","TxStatus":"I"} {"Type":"ReadyForQuery","TxStatus":"I"} -send +send crdb_only Bind {"DestinationPortal": "p7", "PreparedStatement": "s7", "ParameterFormatCodes": [0], "Parameters": [{"text":"T"}]} Execute {"Portal": "p7"} Sync ---- -until +until crdb_only ReadyForQuery ---- {"Type":"BindComplete"} {"Type":"DataRow","Values":[{"text":"52"}]} {"Type":"CommandComplete","CommandTag":"SELECT 1"} {"Type":"ReadyForQuery","TxStatus":"I"} + +send +Query {"String": "DROP TABLE IF EXISTS tab"} +---- + +until ignore=NoticeResponse +ReadyForQuery +---- +{"Type":"CommandComplete","CommandTag":"DROP TABLE"} +{"Type":"ReadyForQuery","TxStatus":"I"} + +send +Query {"String": "BEGIN"} +---- + +until ignore=NoticeResponse +ReadyForQuery +---- +{"Type":"CommandComplete","CommandTag":"BEGIN"} +{"Type":"ReadyForQuery","TxStatus":"T"} + +send +Parse {"Name": "s8", "Query": "SELECT relname FROM pg_class WHERE oid = $1::regclass"} +Bind {"DestinationPortal": "p8", "PreparedStatement": "s8", "ResultFormatCodes": [0], "Parameters": [{"text":"t"}]} +Sync +---- + +until +ReadyForQuery +---- +{"Type":"ParseComplete"} +{"Type":"BindComplete"} +{"Type":"ReadyForQuery","TxStatus":"T"} + +send +Query {"String": "ALTER TABLE t RENAME TO tab"} +---- + +until +ReadyForQuery +---- +{"Type":"CommandComplete","CommandTag":"ALTER TABLE"} +{"Type":"ReadyForQuery","TxStatus":"T"} + +send +Execute {"Portal": "p8"} +Sync +---- + +until noncrdb_only +ReadyForQuery +---- +{"Type":"DataRow","Values":[{"text":"t"}]} +{"Type":"CommandComplete","CommandTag":"SELECT 1"} +{"Type":"ReadyForQuery","TxStatus":"T"} + +# Currently, CRDB differs in that it returns the new table name here, but the +# important part of this test is that it asserts that binding the placeholder +# parameter occurred before the table rename, which matches the Postgres +# behavior. +# TODO(rafi): To be fully correct, we still should return table name 't' here. +until crdb_only +ReadyForQuery +---- +{"Type":"DataRow","Values":[{"text":"tab"}]} +{"Type":"CommandComplete","CommandTag":"SELECT 1"} +{"Type":"ReadyForQuery","TxStatus":"T"} + +send +Query {"String": "COMMIT"} +---- + +until ignore=NoticeResponse +ReadyForQuery +---- +{"Type":"CommandComplete","CommandTag":"COMMIT"} +{"Type":"ReadyForQuery","TxStatus":"I"} diff --git a/pkg/sql/pgwire/testdata/pgtest/enum b/pkg/sql/pgwire/testdata/pgtest/enum index e1d45bb9025e..08736de962c3 100644 --- a/pkg/sql/pgwire/testdata/pgtest/enum +++ b/pkg/sql/pgwire/testdata/pgtest/enum @@ -106,15 +106,30 @@ ReadyForQuery {"Type":"CommandComplete","CommandTag":"INSERT 0 1"} {"Type":"ReadyForQuery","TxStatus":"I"} +send noncrdb_only +Parse {"Name": "s1", "Query": "INSERT INTO tb VALUES ($1)"} +Bind {"DestinationPortal": "p", "PreparedStatement": "s1", "ParameterFormatCodes": [0], "Parameters": [{"text":"hi"}]} +Execute {"Portal": "p"} +Sync +---- + +until noncrdb_only +ReadyForQuery +---- +{"Type":"ParseComplete"} +{"Type":"BindComplete"} +{"Type":"CommandComplete","CommandTag":"INSERT 0 1"} +{"Type":"ReadyForQuery","TxStatus":"I"} + # Ensure that our value was successfully inserted. -send crdb_only +send Query {"String": "SELECT * FROM tb"} ---- -until crdb_only +until ignore_type_oids ignore_table_oids ignore_data_type_sizes ReadyForQuery ---- -{"Type":"RowDescription","Fields":[{"Name":"x","TableOID":54,"TableAttributeNumber":1,"DataTypeOID":100052,"DataTypeSize":-1,"TypeModifier":-1,"Format":0}]} +{"Type":"RowDescription","Fields":[{"Name":"x","TableOID":0,"TableAttributeNumber":1,"DataTypeOID":0,"DataTypeSize":0,"TypeModifier":-1,"Format":0}]} {"Type":"DataRow","Values":[{"text":"hi"}]} {"Type":"CommandComplete","CommandTag":"SELECT 1"} {"Type":"ReadyForQuery","TxStatus":"I"} @@ -193,3 +208,40 @@ ReadyForQuery {"Type":"BindComplete"} {"Type":"CommandComplete","CommandTag":"INSERT 0 1"} {"Type":"ReadyForQuery","TxStatus":"I"} + +send +Query {"String": "ALTER TYPE te ADD VALUE 'hola'"} +---- + +until +ReadyForQuery +---- +{"Type":"CommandComplete","CommandTag":"ALTER TYPE"} +{"Type":"ReadyForQuery","TxStatus":"I"} + +# Reuse original prepared statement now that there's a new enum value. +send +Bind {"DestinationPortal": "p", "PreparedStatement": "s1", "ParameterFormatCodes": [0], "Parameters": [{"text":"hola"}]} +Execute {"Portal": "p"} +Sync +---- + +until +ReadyForQuery +---- +{"Type":"BindComplete"} +{"Type":"CommandComplete","CommandTag":"INSERT 0 1"} +{"Type":"ReadyForQuery","TxStatus":"I"} + +# Ensure that our value was successfully inserted. +send +Query {"String": "SELECT * FROM tb WHERE x = 'hola'"} +---- + +until ignore_type_oids ignore_table_oids ignore_data_type_sizes +ReadyForQuery +---- +{"Type":"RowDescription","Fields":[{"Name":"x","TableOID":0,"TableAttributeNumber":1,"DataTypeOID":0,"DataTypeSize":0,"TypeModifier":-1,"Format":0}]} +{"Type":"DataRow","Values":[{"text":"hola"}]} +{"Type":"CommandComplete","CommandTag":"SELECT 1"} +{"Type":"ReadyForQuery","TxStatus":"I"} diff --git a/pkg/sql/tests/enum_test.go b/pkg/sql/tests/enum_test.go index c8e0d56a4103..f92867ce3f1c 100644 --- a/pkg/sql/tests/enum_test.go +++ b/pkg/sql/tests/enum_test.go @@ -111,9 +111,7 @@ func (i intItem) Less(o btree.Item) bool { } // TestEnumPlaceholderWithAsOfSystemTime is a regression test for an edge case -// with bind where we would not properly deal with leases involving types. At -// the time of writing this test, we still don't deal with such leases properly -// but we did fix any really dangerous hazards. +// with bind where we would not properly deal with leases involving types. func TestEnumPlaceholderWithAsOfSystemTime(t *testing.T) { defer leaktest.AfterTest(t)() @@ -132,11 +130,13 @@ func TestEnumPlaceholderWithAsOfSystemTime(t *testing.T) { // Before the commit which introduced this test, the below statement would // crash the server. q := fmt.Sprintf("SELECT k FROM tab AS OF SYSTEM TIME %s WHERE v = $1", afterInsert) - db.Exec(t, q, "a") + require.Equal(t, [][]string{{"1"}}, db.QueryStr(t, q, "a")) db.Exec(t, "ALTER TYPE typ RENAME VALUE 'a' TO 'd'") db.Exec(t, "ALTER TYPE typ RENAME VALUE 'b' TO 'a'") - got := db.QueryStr(t, q, "a") - require.Equal(t, [][]string{{"1"}}, got) + // The AOST does not apply to the transaction that binds 'a' to the + // placeholder. + require.Equal(t, [][]string{}, db.QueryStr(t, q, "a")) + require.Equal(t, [][]string{{"1"}}, db.QueryStr(t, q, "d")) } // TestEnumDropValueCheckConstraint tests that check constraints containing