Skip to content

Commit

Permalink
sql: add session variable for multiple active portals
Browse files Browse the repository at this point in the history
This change introduces a new session variable for a preview feature. When set to `true`,
all non-internal portals with read-only [`SELECT`](../v23.1/selection-queries.html)
queries without sub-queries or post-queries can be paused and resumed in an interleaving
manner, but are executed with a local plan.

Release note (SQL change): Added the session variable `multiple_active_portals_enabled`.
This setting is only for a preview feature. When set to `true`, it allows
multiple portals to be open at the same time, with their execution interleaved
with each other. In other words, these portals can be paused. The underlying
statement for a pausable portal must be a read-only `SELECT` query without
sub-queries or post-queries, and such a portal is always executed with a local
plan.
  • Loading branch information
ZhouXing19 committed Apr 9, 2023
1 parent 3c5b406 commit 688d398
Show file tree
Hide file tree
Showing 15 changed files with 56 additions and 39 deletions.
1 change: 0 additions & 1 deletion docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
1 change: 0 additions & 1 deletion docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
Expand Up @@ -201,7 +201,6 @@
<tr><td><div id="setting-sql-multiregion-drop-primary-region-enabled" class="anchored"><code>sql.multiregion.drop_primary_region.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>allows dropping the PRIMARY REGION of a database if it is the last region</td></tr>
<tr><td><div id="setting-sql-notices-enabled" class="anchored"><code>sql.notices.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>enable notices in the server/client protocol being sent</td></tr>
<tr><td><div id="setting-sql-optimizer-uniqueness-checks-for-gen-random-uuid-enabled" class="anchored"><code>sql.optimizer.uniqueness_checks_for_gen_random_uuid.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>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</td></tr>
<tr><td><div id="setting-sql-pgwire-multiple-active-portals-enabled" class="anchored"><code>sql.pgwire.multiple_active_portals.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>if true, portals with read-only SELECT query without sub/post queries can be executed in interleaving manner, but with local execution plan</td></tr>
<tr><td><div id="setting-sql-schema-telemetry-recurrence" class="anchored"><code>sql.schema.telemetry.recurrence</code></div></td><td>string</td><td><code>@weekly</code></td><td>cron-tab recurrence for SQL schema telemetry job</td></tr>
<tr><td><div id="setting-sql-show-ranges-deprecated-behavior-enabled" class="anchored"><code>sql.show_ranges_deprecated_behavior.enabled</code></div></td><td>boolean</td><td><code>true</code></td><td>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.</td></tr>
<tr><td><div id="setting-sql-spatial-experimental-box2d-comparison-operators-enabled" class="anchored"><code>sql.spatial.experimental_box2d_comparison_operators.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>enables the use of certain experimental box2d comparison operators</td></tr>
Expand Down
1 change: 0 additions & 1 deletion pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -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",
Expand Down
8 changes: 0 additions & 8 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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"
Expand Down Expand Up @@ -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,
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/distsql_running.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
12 changes: 4 additions & 8 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")

Expand Down Expand Up @@ -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
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/execinfra/base.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -5175,6 +5175,7 @@ WHERE
'distsql_workmem',
'copy_fast_path_enabled',
'direct_columnar_scans_enabled'
'multiple_active_portals_enabled'
);
----
variable value
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/pgwire/testdata/pgtest/portals_crbugs
Original file line number Diff line number Diff line change
Expand Up @@ -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"}

Expand Down Expand Up @@ -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
Expand Down
17 changes: 10 additions & 7 deletions pkg/sql/prepared_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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
)

Expand Down Expand Up @@ -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
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/sessiondatapb/local_only_session_data.proto
Original file line number Diff line number Diff line change
Expand Up @@ -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 //
Expand Down
8 changes: 5 additions & 3 deletions pkg/sql/sqltelemetry/pgwire.go
Original file line number Diff line number Diff line change
Expand Up @@ -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")
17 changes: 17 additions & 0 deletions pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.
Expand Down

0 comments on commit 688d398

Please sign in to comment.