Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
71632: sql: make sure pgwire bind always happens in a transaction r=otan a=rafiss

fixes #70378
and maybe #64140

The approach of using a transaction matches what we do for pgwire Parse
messages already. This is important to make sure that user-defined types
are leased correctly.

This also updated the SQL EXECUTE command to resolve user-defined types
so that it gets the latest changes.

Release note (bug fix): Adding new values to a user-defined enum type
will previously would cause a prepared statement using that type to not
work. This now works as expected.

71836: clusterversion: introduce 22.1 development versions r=celiala a=celiala

Fix: #69828

Looking at past Version values for `Start{XX_X}`, I think the `Version` value for `Key: Start22_1` should instead be `Major: 21, Minor: 2, ...`.


Start21_1 (from #70268):
```
// v21.1 versions. Internal versions defined here-on-forth must be even.
{
  Key:     Start21_1,
  Version: roachpb.Version{Major: 20, Minor: 2, Internal: 2},
},
```

Start 21_2:
```
{
  Key:     Start21_2,
  Version: roachpb.Version{Major: 21, Minor: 1, Internal: 1102},
},
```

Release justification: Non-production code change.
Release note: None

71856: bazel: skip recompilation when using dev+bazel r=irfansharif a=irfansharif

Fixes #71835. When switching between using `dev` and `bazel` raw, I kept
seeing our C++ protobuf dependency getting recompiled (slowly).
It appears that the bazel build for protobuf has a dependency on $PATH
(see bazelbuild/intellij#1169 and bazelbuild/bazel#7095). Specifying
[`--incompatible_strict_action_env`](https://docs.bazel.build/versions/main/command-line-reference.html#flag--incompatible_strict_action_env) pins PATH and avoids the build cache
thrashing we were seeing before.

Release note: None

Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Celia La <celiala456@gmail.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
4 people committed Oct 22, 2021
4 parents fb7bbfc + 01ca1e5 + 3b3bec8 + ff59b0d commit df06437
Show file tree
Hide file tree
Showing 13 changed files with 248 additions and 59 deletions.
4 changes: 3 additions & 1 deletion .bazelrc
Original file line number Diff line number Diff line change
@@ -1,6 +1,6 @@
try-import %workspace%/.bazelrc.user

build --symlink_prefix=_bazel/ --ui_event_filters=-DEBUG --define gotags=bazel,crdb_test_off,gss --experimental_proto_descriptor_sets_include_source_info
build --symlink_prefix=_bazel/ --ui_event_filters=-DEBUG --define gotags=bazel,crdb_test_off,gss --experimental_proto_descriptor_sets_include_source_info --incompatible_strict_action_env
test --config=test
build:with_ui --define cockroach_with_ui=y
build:test --define gotags=bazel,crdb_test,gss
Expand Down Expand Up @@ -48,3 +48,5 @@ build:devdarwinx86_64 --crosstool_top=@toolchain_dev_darwin_x86-64//:suite
build:devdarwinx86_64 --config=dev
build:dev --define cockroach_bazel_dev=y
build:dev --stamp --workspace_status_command=./build/bazelutil/stamp.sh

# vi: ft=sh
2 changes: 1 addition & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -166,4 +166,4 @@ trace.debug.enable boolean false if set, traces for recent requests can be seen
trace.jaeger.agent string the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.
trace.opentelemetry.collector string address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.
trace.zipkin.collector string the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.
version version 22.1-102 set the active cluster version in the format '<major>.<minor>'
version version 21.2-4 set the active cluster version in the format '<major>.<minor>'
2 changes: 1 addition & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -171,6 +171,6 @@
<tr><td><code>trace.jaeger.agent</code></td><td>string</td><td><code></code></td><td>the address of a Jaeger agent to receive traces using the Jaeger UDP Thrift protocol, as <host>:<port>. If no port is specified, 6381 will be used.</td></tr>
<tr><td><code>trace.opentelemetry.collector</code></td><td>string</td><td><code></code></td><td>address of an OpenTelemetry trace collector to receive traces using the otel gRPC protocol, as <host>:<port>. If no port is specified, 4317 will be used.</td></tr>
<tr><td><code>trace.zipkin.collector</code></td><td>string</td><td><code></code></td><td>the address of a Zipkin instance to receive traces, as <host>:<port>. If no port is specified, 9411 will be used.</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>22.1-102</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
<tr><td><code>version</code></td><td>version</td><td><code>21.2-4</code></td><td>set the active cluster version in the format '<major>.<minor>'</td></tr>
</tbody>
</table>
10 changes: 5 additions & 5 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,8 +184,8 @@ const (
PostTruncatedAndRangeAppliedStateMigration
// V21_1 is CockroachDB v21.1. It's used for all v21.1.x patch releases.
//
// TODO(irfansharif): This can be removed as part of #69828 (bumping the min
// cluster version).
// TODO(irfansharif): This can be removed as part of #71708 (bump
// min-supported version to 21.2).
V21_1

// v21.1PLUS release. This is a special v21.1.x release with extra changes,
Expand Down Expand Up @@ -506,14 +506,14 @@ var versionsSingleton = keyedVersions{
Version: roachpb.Version{Major: 21, Minor: 2},
},

// v22.1 versions.
// v22.1 versions. Internal versions must be even.
{
Key: Start22_1,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 100},
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2},
},
{
Key: TargetBytesAvoidExcess,
Version: roachpb.Version{Major: 22, Minor: 1, Internal: 102},
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 4},
},

// *************************************************
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down
76 changes: 53 additions & 23 deletions pkg/sql/conn_executor_prepare.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}

Expand Down
19 changes: 12 additions & 7 deletions pkg/sql/execute.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,20 +17,15 @@ 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"
)

// fillInPlaceholder helps with the EXECUTE foo(args) SQL statement: it takes in
// 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,
Expand All @@ -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,
)
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal
Original file line number Diff line number Diff line change
Expand Up @@ -380,7 +380,7 @@ select crdb_internal.get_vmodule()
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
----
22.1
21.2

query ITTT colnames
select node_id, component, field, regexp_replace(regexp_replace(value, '^\d+$', '<port>'), e':\\d+', ':<port>') as value from crdb_internal.node_runtime_info
Expand Down Expand Up @@ -539,7 +539,7 @@ select * from crdb_internal.node_inflight_trace_spans
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
----
22.1
21.2

user root

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -350,7 +350,7 @@ select crdb_internal.get_vmodule()
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
----
22.1
21.2

query ITTT colnames
select node_id, component, field, regexp_replace(regexp_replace(value, '^\d+$', '<port>'), e':\\d+', ':<port>') as value from crdb_internal.node_runtime_info
Expand Down Expand Up @@ -443,7 +443,7 @@ select * from crdb_internal.gossip_alerts
query T
select regexp_replace(crdb_internal.node_executable_version()::string, '(-\d+)?$', '');
----
22.1
21.2

user root

Expand Down
24 changes: 24 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/enums
Original file line number Diff line number Diff line change
Expand Up @@ -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
90 changes: 83 additions & 7 deletions pkg/sql/pgwire/testdata/pgtest/bind_and_resolve
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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"}
Loading

0 comments on commit df06437

Please sign in to comment.