diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index c376c084e63a..9865a8e72109 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -250,7 +250,6 @@ sql.multiple_modifications_of_table.enabled boolean false if true, allow stateme
sql.multiregion.drop_primary_region.enabled boolean true allows dropping the PRIMARY REGION of a database if it is the last region
sql.notices.enabled boolean true enable notices in the server/client protocol being sent
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled boolean false if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability
-sql.pgwire.multiple_active_portals.enabled boolean false if true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan
sql.schema.telemetry.recurrence string @weekly cron-tab recurrence for SQL schema telemetry job
sql.show_ranges_deprecated_behavior.enabled boolean true if set, SHOW RANGES and crdb_internal.ranges{_no_leases} behave with deprecated pre-v23.1 semantics. NB: the new SHOW RANGES interface has richer WITH options than pre-v23.1 SHOW RANGES.
sql.spatial.experimental_box2d_comparison_operators.enabled boolean false enables the use of certain experimental box2d comparison operators
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 1cbdd6b609c3..09384a4f8eff 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -201,7 +201,6 @@
sql.multiregion.drop_primary_region.enabled
| boolean | true | allows dropping the PRIMARY REGION of a database if it is the last region |
sql.notices.enabled
| boolean | true | enable notices in the server/client protocol being sent |
sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled
| boolean | false | if enabled, uniqueness checks may be planned for mutations of UUID columns updated with gen_random_uuid(); otherwise, uniqueness is assumed due to near-zero collision probability |
-sql.pgwire.multiple_active_portals.enabled
| boolean | false | if true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan |
sql.schema.telemetry.recurrence
| string | @weekly | cron-tab recurrence for SQL schema telemetry job |
sql.show_ranges_deprecated_behavior.enabled
| boolean | true | if set, SHOW RANGES and crdb_internal.ranges{_no_leases} behave with deprecated pre-v23.1 semantics. NB: the new SHOW RANGES interface has richer WITH options than pre-v23.1 SHOW RANGES. |
sql.spatial.experimental_box2d_comparison_operators.enabled
| boolean | false | enables the use of certain experimental box2d comparison operators |
diff --git a/pkg/server/BUILD.bazel b/pkg/server/BUILD.bazel
index fc03ce4c7f81..5cecaed24f56 100644
--- a/pkg/server/BUILD.bazel
+++ b/pkg/server/BUILD.bazel
@@ -251,7 +251,6 @@ go_library(
"//pkg/sql/sqlstats/insights",
"//pkg/sql/sqlstats/persistedsqlstats",
"//pkg/sql/sqlstats/persistedsqlstats/sqlstatsutil",
- "//pkg/sql/sqltelemetry",
"//pkg/sql/stats",
"//pkg/sql/stmtdiagnostics",
"//pkg/sql/syntheticprivilege",
diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go
index 262e4d953964..1989f016dcdf 100644
--- a/pkg/server/server_sql.go
+++ b/pkg/server/server_sql.go
@@ -59,7 +59,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/server/settingswatcher"
"github.com/cockroachdb/cockroach/pkg/server/status"
"github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher"
- "github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/server/tracedumper"
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
@@ -103,7 +102,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slinstance"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slprovider"
"github.com/cockroachdb/cockroach/pkg/sql/sqlstats"
- "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/stats"
"github.com/cockroachdb/cockroach/pkg/sql/stmtdiagnostics"
"github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilegecache"
@@ -1353,12 +1351,6 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) {
vmoduleSetting.SetOnChange(&cfg.Settings.SV, fn)
fn(ctx)
- sql.EnableMultipleActivePortals.SetOnChange(&cfg.Settings.SV, func(ctx context.Context) {
- if sql.EnableMultipleActivePortals.Get(&cfg.Settings.SV) {
- telemetry.Inc(sqltelemetry.MultipleActivePortalCounter)
- }
- })
-
return &SQLServer{
ambientCtx: cfg.BaseConfig.AmbientCtx,
stopper: cfg.stopper,
diff --git a/pkg/sql/distsql_running.go b/pkg/sql/distsql_running.go
index 5c89251d216e..34ea495ad917 100644
--- a/pkg/sql/distsql_running.go
+++ b/pkg/sql/distsql_running.go
@@ -1549,16 +1549,16 @@ func (r *DistSQLReceiver) PushBatch(
var (
// ErrLimitedResultNotSupported is an error produced by pgwire
// indicating the user attempted to have multiple active portals but
- // either without setting sql.pgwire.multiple_active_portals.enabled to
+ // either without setting session variable multiple_active_portals_enabled to
// true or the underlying query does not satisfy the restriction.
ErrLimitedResultNotSupported = unimplemented.NewWithIssue(
40195,
"multiple active portals not supported, "+
- "please set sql.pgwire.multiple_active_portals.enabled to true. "+
+ "please set session variable multiple_active_portals_enabled to true. "+
"Note: this feature is in preview",
)
// ErrStmtNotSupportedForPausablePortal is returned when the user have set
- // sql.pgwire.multiple_active_portals.enabled to true but set an unsupported
+ // session variable multiple_active_portals_enabled to true but set an unsupported
// statement for a portal.
ErrStmtNotSupportedForPausablePortal = unimplemented.NewWithIssue(
98911,
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index 4aebccb2298e..aa7db94e1826 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -733,14 +733,6 @@ var overrideAlterPrimaryRegionInSuperRegion = settings.RegisterBoolSetting(
false,
).WithPublic()
-var EnableMultipleActivePortals = settings.RegisterBoolSetting(
- settings.TenantWritable,
- "sql.pgwire.multiple_active_portals.enabled",
- "if true, portals with read-only SELECT query without sub/post queries "+
- "can be executed in interleaving manner, but with local execution plan",
- false,
-).WithPublic()
-
var errNoTransactionInProgress = errors.New("there is no transaction in progress")
var errTransactionInProgress = errors.New("there is already a transaction in progress")
@@ -3536,6 +3528,10 @@ func (m *sessionDataMutator) SetStreamerEnabled(val bool) {
m.data.StreamerEnabled = val
}
+func (m *sessionDataMutator) SetMultipleActivePortalsEnabled(val bool) {
+ m.data.MultipleActivePortalsEnabled = val
+}
+
// Utility functions related to scrubbing sensitive information on SQL Stats.
// quantizeCounts ensures that the Count field in the
diff --git a/pkg/sql/execinfra/base.go b/pkg/sql/execinfra/base.go
index 496e6bce51fa..737302352526 100644
--- a/pkg/sql/execinfra/base.go
+++ b/pkg/sql/execinfra/base.go
@@ -43,8 +43,9 @@ const (
NeedMoreRows ConsumerStatus = iota
// SwitchToAnotherPortal indicates that the we received exec command for
// a different portal, and may come back to continue executing the current
- // portal later. If the cluster setting sql.pgwire.multiple_active_portals.enabled
- // is set to be true, we do nothing and return the control to the connExecutor.
+ // portal later. If the cluster setting session variable
+ // multiple_active_portals_enabled is set to be true, we do nothing and return
+ // the control to the connExecutor.
SwitchToAnotherPortal
// DrainRequested indicates that the consumer will not process any more data
// rows, but will accept trailing metadata from the producer.
diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema
index 1e1852679121..ec2c19bd3d8c 100644
--- a/pkg/sql/logictest/testdata/logic_test/information_schema
+++ b/pkg/sql/logictest/testdata/logic_test/information_schema
@@ -5175,6 +5175,7 @@ WHERE
'distsql_workmem',
'copy_fast_path_enabled',
'direct_columnar_scans_enabled'
+ 'multiple_active_portals_enabled'
);
----
variable value
@@ -5207,6 +5208,7 @@ default_transaction_read_only off
default_transaction_use_follower_reads off
default_with_oids off
descriptor_validation on
+direct_columnar_scans_enabled on
disable_drop_tenant off
disable_hoist_projection_in_join_limitation off
disable_partially_distributed_plans off
@@ -5265,6 +5267,7 @@ lock_timeout 0
log_timezone UTC
max_identifier_length 128
max_index_keys 32
+multiple_active_portals_enabled off
node_id 1
null_ordered_last off
on_update_rehome_row_enabled on
diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog
index e6e79c34cffe..c343895e445d 100644
--- a/pkg/sql/logictest/testdata/logic_test/pg_catalog
+++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog
@@ -2669,7 +2669,7 @@ SELECT
FROM
pg_catalog.pg_settings
WHERE
- name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled')
+ name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled', 'multiple_active_portals_enabled')
----
name setting category short_desc extra_desc vartype
allow_ordinal_column_references off NULL NULL NULL string
@@ -2822,7 +2822,7 @@ SELECT
FROM
pg_catalog.pg_settings
WHERE
- name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled')
+ name NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled', 'multiple_active_portals_enabled')
----
name setting unit context enumvals boot_val reset_val
allow_ordinal_column_references off NULL user NULL off off
@@ -3058,6 +3058,7 @@ lock_timeout NULL NULL NULL
log_timezone NULL NULL NULL NULL NULL
max_identifier_length NULL NULL NULL NULL NULL
max_index_keys NULL NULL NULL NULL NULL
+multiple_active_portals_enabled NULL NULL NULL NULL NULL
node_id NULL NULL NULL NULL NULL
null_ordered_last NULL NULL NULL NULL NULL
on_update_rehome_row_enabled NULL NULL NULL NULL NULL
diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source
index 5222c85b06c7..60f610cd5e99 100644
--- a/pkg/sql/logictest/testdata/logic_test/show_source
+++ b/pkg/sql/logictest/testdata/logic_test/show_source
@@ -23,7 +23,7 @@ UTF8 1
query TT colnames
SELECT *
FROM [SHOW ALL]
-WHERE variable NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled')
+WHERE variable NOT IN ('optimizer', 'crdb_version', 'session_id', 'distsql_workmem', 'copy_fast_path_enabled', 'direct_columnar_scans_enabled', 'multiple_active_portals_enabled')
----
variable value
allow_ordinal_column_references off
diff --git a/pkg/sql/pgwire/testdata/pgtest/portals_crbugs b/pkg/sql/pgwire/testdata/pgtest/portals_crbugs
index 1afeb7f1788f..14211a72b413 100644
--- a/pkg/sql/pgwire/testdata/pgtest/portals_crbugs
+++ b/pkg/sql/pgwire/testdata/pgtest/portals_crbugs
@@ -29,7 +29,7 @@ ReadyForQuery
{"Type":"BindComplete"}
{"Type":"DataRow","Values":[{"text":"1"}]}
{"Type":"PortalSuspended"}
-{"Type":"ErrorResponse","Code":"0A000","Message":"unimplemented: multiple active portals not supported, please set sql.pgwire.multiple_active_portals.enabled to true. Note: this feature is in preview"}
+{"Type":"ErrorResponse","Code":"0A000","Message":"unimplemented: multiple active portals not supported, please set session variable multiple_active_portals_enabled to true. Note: this feature is in preview"}
{"Type":"ReadyForQuery","TxStatus":"E"}
{"Type":"ReadyForQuery","TxStatus":"E"}
@@ -70,7 +70,7 @@ ReadyForQuery
{"Type":"BindComplete"}
{"Type":"DataRow","Values":[{"text":"1"}]}
{"Type":"PortalSuspended"}
-{"Type":"ErrorResponse","Code":"0A000","Message":"unimplemented: multiple active portals not supported, please set sql.pgwire.multiple_active_portals.enabled to true. Note: this feature is in preview"}
+{"Type":"ErrorResponse","Code":"0A000","Message":"unimplemented: multiple active portals not supported, please set session variable multiple_active_portals_enabled to true. Note: this feature is in preview"}
{"Type":"ReadyForQuery","TxStatus":"E"}
send
diff --git a/pkg/sql/prepared_stmt.go b/pkg/sql/prepared_stmt.go
index 2f3834bda37c..aa3115adc405 100644
--- a/pkg/sql/prepared_stmt.go
+++ b/pkg/sql/prepared_stmt.go
@@ -15,11 +15,13 @@ import (
"time"
"unsafe"
+ "github.com/cockroachdb/cockroach/pkg/server/telemetry"
"github.com/cockroachdb/cockroach/pkg/sql/flowinfra"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgwirebase"
"github.com/cockroachdb/cockroach/pkg/sql/querycache"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
+ "github.com/cockroachdb/cockroach/pkg/sql/sqltelemetry"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/mon"
@@ -127,15 +129,15 @@ type PortalPausablity int64
const (
// PortalPausabilityDisabled is the default status of a portal when
- // sql.pgwire.multiple_active_portals.enabled is false.
+ // the session variable multiple_active_portals_enabled is false.
PortalPausabilityDisabled PortalPausablity = iota
- // PausablePortal is set when sql.pgwire.multiple_active_portals.enabled is
- // set to true and the underlying statement is a read-only SELECT query with
- // no sub-queries or post-queries.
+ // PausablePortal is set when the session variable multiple_active_portals_enabled
+ // is set to true and the underlying statement is a read-only SELECT query
+ // with no sub-queries or post-queries.
PausablePortal
// NotPausablePortalForUnsupportedStmt is used when the cluster setting
- // sql.pgwire.multiple_active_portals.enabled is set to true, while we don't
- // support underlying statement.
+ // the session variable multiple_active_portals_enabled is set to true, while
+ // we don't support underlying statement.
NotPausablePortalForUnsupportedStmt
)
@@ -180,7 +182,8 @@ func (ex *connExecutor) makePreparedPortal(
OutFormats: outFormats,
}
- if EnableMultipleActivePortals.Get(&ex.server.cfg.Settings.SV) {
+ if ex.sessionData().MultipleActivePortalsEnabled {
+ telemetry.Inc(sqltelemetry.StmtsTriedWithPausablePortals)
portal.pauseInfo = &portalPauseInfo{}
portal.portalPausablity = PausablePortal
}
diff --git a/pkg/sql/sessiondatapb/local_only_session_data.proto b/pkg/sql/sessiondatapb/local_only_session_data.proto
index 9a5fd41d163d..19914953017a 100644
--- a/pkg/sql/sessiondatapb/local_only_session_data.proto
+++ b/pkg/sql/sessiondatapb/local_only_session_data.proto
@@ -369,6 +369,11 @@ message LocalOnlySessionData {
// DisableDropTenant causes errors when the client
// attempts to drop tenants or tenant records.
bool disable_drop_tenant = 99;
+ // MultipleActivePortalEnabled determines if the pgwire portal execution
+ // for certain queries can be paused. If true, portals with read-only SELECT
+ // query without sub/post queries can be executed in interleaving manner, but
+ // with a local execution plan.
+ bool multiple_active_portals_enabled = 100;
///////////////////////////////////////////////////////////////////////////
// WARNING: consider whether a session parameter you're adding needs to //
diff --git a/pkg/sql/sqltelemetry/pgwire.go b/pkg/sql/sqltelemetry/pgwire.go
index 18a7ef68e867..3cacbfad4cdd 100644
--- a/pkg/sql/sqltelemetry/pgwire.go
+++ b/pkg/sql/sqltelemetry/pgwire.go
@@ -69,6 +69,8 @@ var CloseRequestCounter = telemetry.GetCounterOnce("pgwire.command.close")
// is made.
var FlushRequestCounter = telemetry.GetCounterOnce("pgwire.command.flush")
-// MultipleActivePortalCounter is to be incremented every time the cluster setting
-// sql.pgwire.multiple_active_portals.enabled is set true.
-var MultipleActivePortalCounter = telemetry.GetCounterOnce("pgwire.multiple_active_portals")
+// StmtsTriedWithPausablePortals is to be incremented every time there's a
+// not-internal statement executed with a pgwire portal and the session variable
+// multiple_active_portals_enabled has been set to true.
+// The statement might not satisfy the restriction for a pausable portal.
+var StmtsTriedWithPausablePortals = telemetry.GetCounterOnce("pgwire.pausable_portal_stmts")
diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go
index e5560b65ee2c..0b9025e3cf24 100644
--- a/pkg/sql/vars.go
+++ b/pkg/sql/vars.go
@@ -2648,6 +2648,23 @@ var varGen = map[string]sessionVar{
return formatBoolAsPostgresSetting(execinfra.UseStreamerEnabled.Get(sv))
},
},
+
+ // CockroachDB extension.
+ `multiple_active_portals_enabled`: {
+ GetStringVal: makePostgresBoolGetStringValFn(`multiple_active_portals_enabled`),
+ Set: func(_ context.Context, m sessionDataMutator, s string) error {
+ b, err := paramparse.ParseBoolVar("multiple_active_portals_enabled", s)
+ if err != nil {
+ return err
+ }
+ m.SetMultipleActivePortalsEnabled(b)
+ return nil
+ },
+ Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) {
+ return formatBoolAsPostgresSetting(evalCtx.SessionData().MultipleActivePortalsEnabled), nil
+ },
+ GlobalDefault: globalFalse,
+ },
}
// We want test coverage for this on and off so make it metamorphic.