Skip to content

Commit

Permalink
server,db-console: add feature flag for Cross-Cluster Replication das…
Browse files Browse the repository at this point in the history
…hboard

This change adds a feature flag whose value is determined by the cluster
setting `cross_cluster_replication.enabled`. This feature flag is then used
when rendering the metrics dashboard dropdown options to decide whether to
display the cross-cluster replication dashboard.

This change also deletes the session variable and associated cluster setting
`sql.defaults.experimental_stream_replication.enabled` so as to unify all
interactions with cross-cluster replication under a single cluster setting.

Fixes: #95995

Release note: None
  • Loading branch information
adityamaru committed Feb 7, 2023
1 parent 7a19bf6 commit bbc6ec4
Show file tree
Hide file tree
Showing 20 changed files with 63 additions and 53 deletions.
3 changes: 0 additions & 3 deletions docs/generated/settings/settings-for-tenants.txt
Original file line number Diff line number Diff line change
Expand Up @@ -134,9 +134,6 @@ This session variable default should now be configured using ALTER ROLE... SET:
sql.defaults.experimental_implicit_column_partitioning.enabled boolean false "default value for experimental_enable_temp_tables; allows for the use of implicit column partitioning
This cluster setting is being kept to preserve backwards-compatibility.
This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html"
sql.defaults.experimental_stream_replication.enabled boolean false "default value for experimental_stream_replication session setting;enables the ability to setup a replication stream
This cluster setting is being kept to preserve backwards-compatibility.
This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html"
sql.defaults.experimental_temporary_tables.enabled boolean false "default value for experimental_enable_temp_tables; allows for use of temporary tables by default
This cluster setting is being kept to preserve backwards-compatibility.
This session variable default should now be configured using ALTER ROLE... SET: https://www.cockroachlabs.com/docs/stable/alter-role.html"
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 @@ -135,7 +135,6 @@
<tr><td><div id="setting-sql-defaults-experimental-distsql-planning" class="anchored"><code>sql.defaults.experimental_distsql_planning</code></div></td><td>enumeration</td><td><code>off</code></td><td>default experimental_distsql_planning mode; enables experimental opt-driven DistSQL planning [off = 0, on = 1]<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
<tr><td><div id="setting-sql-defaults-experimental-enable-unique-without-index-constraints-enabled" class="anchored"><code>sql.defaults.experimental_enable_unique_without_index_constraints.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>default value for experimental_enable_unique_without_index_constraints session setting;disables unique without index constraints by default<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
<tr><td><div id="setting-sql-defaults-experimental-implicit-column-partitioning-enabled" class="anchored"><code>sql.defaults.experimental_implicit_column_partitioning.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>default value for experimental_enable_temp_tables; allows for the use of implicit column partitioning<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
<tr><td><div id="setting-sql-defaults-experimental-stream-replication-enabled" class="anchored"><code>sql.defaults.experimental_stream_replication.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>default value for experimental_stream_replication session setting;enables the ability to setup a replication stream<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
<tr><td><div id="setting-sql-defaults-experimental-temporary-tables-enabled" class="anchored"><code>sql.defaults.experimental_temporary_tables.enabled</code></div></td><td>boolean</td><td><code>false</code></td><td>default value for experimental_enable_temp_tables; allows for use of temporary tables by default<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
<tr><td><div id="setting-sql-defaults-foreign-key-cascades-limit" class="anchored"><code>sql.defaults.foreign_key_cascades_limit</code></div></td><td>integer</td><td><code>10000</code></td><td>default value for foreign_key_cascades_limit session setting; limits the number of cascading operations that run as part of a single query<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
<tr><td><div id="setting-sql-defaults-idle-in-session-timeout" class="anchored"><code>sql.defaults.idle_in_session_timeout</code></div></td><td>duration</td><td><code>0s</code></td><td>default value for the idle_in_session_timeout; default value for the idle_in_session_timeout session setting; controls the duration a session is permitted to idle before the session is terminated; if set to 0, there is no timeout<br/>This cluster setting is being kept to preserve backwards-compatibility.<br/>This session variable default should now be configured using <a href="alter-role.html"><code>ALTER ROLE... SET</code></a></td></tr>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -223,7 +223,7 @@ func NewReplicationHelper(
SET CLUSTER SETTING kv.rangefeed.enabled = true;
SET CLUSTER SETTING kv.closed_timestamp.target_duration = '1s';
SET CLUSTER SETTING changefeed.experimental_poll_interval = '10ms';
SET CLUSTER SETTING sql.defaults.experimental_stream_replication.enabled = 'on';
SET CLUSTER SETTING cross_cluster_replication.enabled = true;
`, `;`)...)

// Sink to read data from.
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/streamingccl/replicationtestutils/testutils.go
Original file line number Diff line number Diff line change
Expand Up @@ -273,7 +273,7 @@ func CreateTenantStreamingClusters(
args.DestInitFunc(t, tsc.DestSysSQL)
}
// Enable stream replication on dest by default.
tsc.DestSysSQL.Exec(t, `SET enable_experimental_stream_replication = true;`)
tsc.DestSysSQL.Exec(t, `SET CLUSTER SETTING cross_cluster_replication.enabled = true;`)
return tsc, func() {
require.NoError(t, srcTenantConn.Close())
destCleanup()
Expand Down
9 changes: 9 additions & 0 deletions pkg/ccl/streamingccl/settings.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,15 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
)

// CrossClusterReplicationEnabled enables the ability to setup and control a
// cross cluster replication stream.
var CrossClusterReplicationEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"cross_cluster_replication.enabled",
"enables the ability to setup and control a cross cluster replication stream",
false,
)

// StreamReplicationStreamLivenessTrackFrequency controls frequency to check
// the liveness of a streaming replication producer job.
var StreamReplicationStreamLivenessTrackFrequency = settings.RegisterDurationSetting(
Expand Down
14 changes: 14 additions & 0 deletions pkg/ccl/streamingccl/streamingest/alter_replication_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (
"context"
"math"

"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl"
"github.com/cockroachdb/cockroach/pkg/ccl/streamingccl/replicationutils"
"github.com/cockroachdb/cockroach/pkg/ccl/utilccl"
"github.com/cockroachdb/cockroach/pkg/jobs"
Expand Down Expand Up @@ -111,6 +112,19 @@ func alterReplicationJobHook(
return nil, nil, nil, false, nil
}

if !streamingccl.CrossClusterReplicationEnabled.Get(&p.ExecCfg().Settings.SV) {
return nil, nil, nil, false, errors.WithTelemetry(
pgerror.WithCandidateCode(
errors.WithHint(
errors.Newf("cross cluster replication is disabled"),
"You can enable cross cluster replication by running `SET CLUSTER SETTING cross_cluster_replication.enabled = true`.",
),
pgcode.ExperimentalFeature,
),
"cross_cluster_replication.enabled",
)
}

if err := p.RequireAdminRole(ctx, "ALTER TENANT REPLICATION"); err != nil {
return nil, nil, nil, false, err
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,7 @@ func TestAlterTenantPauseResume(t *testing.T) {
})

t.Run("pause-resume-as-non-system-tenant", func(t *testing.T) {
c.DestTenantSQL.Exec(t, `SET CLUSTER SETTING cross_cluster_replication.enabled = true`)
c.DestTenantSQL.ExpectErr(t, "only the system tenant can alter tenant", `ALTER TENANT $1 PAUSE REPLICATION`, "foo")
c.DestTenantSQL.ExpectErr(t, "only the system tenant can alter tenant", `ALTER TENANT $1 RESUME REPLICATION`, "foo")
})
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -238,9 +238,9 @@ func TestStreamIngestionJobWithRandomClient(t *testing.T) {

// Attempt to run the ingestion job without enabling the experimental setting.
_, err = conn.Exec(query)
require.True(t, testutils.IsError(err, "stream replication is only supported experimentally"))
require.True(t, testutils.IsError(err, "cross cluster replication is disabled"))

_, err = conn.Exec(`SET enable_experimental_stream_replication = true`)
_, err = conn.Exec(`SET CLUSTER SETTING cross_cluster_replication.enabled = true;`)
require.NoError(t, err)

_, err = conn.Exec(query)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -113,7 +113,7 @@ SET CLUSTER SETTING stream_replication.consumer_heartbeat_frequency = '100ms';
SET CLUSTER SETTING bulkio.stream_ingestion.minimum_flush_interval = '500ms';
SET CLUSTER SETTING bulkio.stream_ingestion.cutover_signal_poll_interval = '100ms';
SET CLUSTER SETTING stream_replication.job_checkpoint_frequency = '100ms';
SET enable_experimental_stream_replication = true;
SET CLUSTER SETTING cross_cluster_replication.enabled = true;
`,
";")...)

Expand Down Expand Up @@ -191,7 +191,7 @@ func TestTenantStreamingCreationErrors(t *testing.T) {
SrcSysSQL.Exec(t, `SET CLUSTER SETTING kv.rangefeed.enabled = true`)

DestSysSQL := sqlutils.MakeSQLRunner(destDB)
DestSysSQL.Exec(t, `SET enable_experimental_stream_replication = true`)
DestSysSQL.Exec(t, `SET CLUSTER SETTING cross_cluster_replication.enabled = true;`)

// Sink to read data from.
srcPgURL, cleanupSink := sqlutils.PGUrl(t, srcServer.ServingSQLAddr(), t.Name(), url.User(username.RootUser))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -68,17 +68,16 @@ func ingestionPlanHook(
return nil, nil, nil, false, nil
}

// Check if the experimental feature is enabled.
if !p.SessionData().EnableStreamReplication {
if !streamingccl.CrossClusterReplicationEnabled.Get(&p.ExecCfg().Settings.SV) {
return nil, nil, nil, false, errors.WithTelemetry(
pgerror.WithCandidateCode(
errors.WithHint(
errors.Newf("stream replication is only supported experimentally"),
"You can enable stream replication by running `SET enable_experimental_stream_replication = true`.",
errors.Newf("cross cluster replication is disabled"),
"You can enable cross cluster replication by running `SET CLUSTER SETTING cross_cluster_replication.enabled = true`.",
),
pgcode.ExperimentalFeature,
),
"replication.ingest.disabled",
"cross_cluster_replication.enabled",
)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -176,7 +176,7 @@ func startReplication(
conn, err := pgx.ConnectConfig(queryCtx, pgxConfig)
require.NoError(t, err)

rows, err := conn.Query(queryCtx, `SET enable_experimental_stream_replication = true`)
rows, err := conn.Query(queryCtx, `SET CLUSTER SETTING cross_cluster_replication.enabled = true;`)
require.NoError(t, err)
rows.Close()

Expand Down
2 changes: 1 addition & 1 deletion pkg/cmd/roachtest/tests/cluster_to_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -611,7 +611,7 @@ func srcClusterSettings(t test.Test, db *sqlutils.SQLRunner) {
}

func destClusterSettings(t test.Test, db *sqlutils.SQLRunner) {
db.ExecMultiple(t, `SET enable_experimental_stream_replication = true;`)
db.ExecMultiple(t, `SET CLUSTER SETTING cross_cluster_replication.enabled = true;`)
}

func copyPGCertsAndMakeURL(
Expand Down
4 changes: 2 additions & 2 deletions pkg/settings/registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -160,7 +160,8 @@ var retiredSettings = map[string]struct{}{
"changefeed.active_protected_timestamps.enabled": {},
"jobs.scheduler.single_node_scheduler.enabled": {},
// renamed.
"spanconfig.host_coalesce_adjacent.enabled": {},
"spanconfig.host_coalesce_adjacent.enabled": {},
"sql.defaults.experimental_stream_replication.enabled": {},
}

// sqlDefaultSettings is the list of "grandfathered" existing sql.defaults
Expand All @@ -182,7 +183,6 @@ var sqlDefaultSettings = map[string]struct{}{
"sql.defaults.experimental_distsql_planning": {},
"sql.defaults.experimental_enable_unique_without_index_constraints.enabled": {},
"sql.defaults.experimental_implicit_column_partitioning.enabled": {},
"sql.defaults.experimental_stream_replication.enabled": {},
"sql.defaults.experimental_temporary_tables.enabled": {},
"sql.defaults.foreign_key_cascades_limit": {},
"sql.defaults.idle_in_session_timeout": {},
Expand Down
8 changes: 0 additions & 8 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -512,14 +512,6 @@ var experimentalUseNewSchemaChanger = settings.RegisterEnumSetting(
},
).WithPublic()

var experimentalStreamReplicationEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"sql.defaults.experimental_stream_replication.enabled",
"default value for experimental_stream_replication session setting;"+
"enables the ability to setup a replication stream",
false,
).WithPublic()

var stubCatalogTablesEnabledClusterValue = settings.RegisterBoolSetting(
settings.TenantWritable,
`sql.defaults.stub_catalog_tables.enabled`,
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/information_schema
Original file line number Diff line number Diff line change
Expand Up @@ -4931,7 +4931,6 @@ disallow_full_table_scans off
enable_auto_rehoming off
enable_drop_enum_value on
enable_experimental_alter_column_type_general off
enable_experimental_stream_replication off
enable_implicit_select_for_update on
enable_implicit_transaction_for_batch_statements on
enable_insert_fast_path on
Expand Down
3 changes: 0 additions & 3 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -2570,7 +2570,6 @@ disallow_full_table_scans off NULL
distsql off NULL NULL NULL string
enable_auto_rehoming off NULL NULL NULL string
enable_experimental_alter_column_type_general off NULL NULL NULL string
enable_experimental_stream_replication off NULL NULL NULL string
enable_implicit_select_for_update on NULL NULL NULL string
enable_implicit_transaction_for_batch_statements on NULL NULL NULL string
enable_insert_fast_path on NULL NULL NULL string
Expand Down Expand Up @@ -2714,7 +2713,6 @@ disallow_full_table_scans off NULL
distsql off NULL user NULL off off
enable_auto_rehoming off NULL user NULL off off
enable_experimental_alter_column_type_general off NULL user NULL off off
enable_experimental_stream_replication off NULL user NULL off off
enable_implicit_select_for_update on NULL user NULL on on
enable_implicit_transaction_for_batch_statements on NULL user NULL on on
enable_insert_fast_path on NULL user NULL on on
Expand Down Expand Up @@ -2856,7 +2854,6 @@ distsql NULL NULL NULL
distsql_workmem NULL NULL NULL NULL NULL
enable_auto_rehoming NULL NULL NULL NULL NULL
enable_experimental_alter_column_type_general NULL NULL NULL NULL NULL
enable_experimental_stream_replication NULL NULL NULL NULL NULL
enable_implicit_select_for_update NULL NULL NULL NULL NULL
enable_implicit_transaction_for_batch_statements NULL NULL NULL NULL NULL
enable_insert_fast_path NULL NULL NULL NULL NULL
Expand Down
1 change: 0 additions & 1 deletion pkg/sql/logictest/testdata/logic_test/show_source
Original file line number Diff line number Diff line change
Expand Up @@ -57,7 +57,6 @@ disallow_full_table_scans off
distsql off
enable_auto_rehoming off
enable_experimental_alter_column_type_general off
enable_experimental_stream_replication off
enable_implicit_select_for_update on
enable_implicit_transaction_for_batch_statements on
enable_insert_fast_path on
Expand Down
18 changes: 0 additions & 18 deletions pkg/sql/vars.go
Original file line number Diff line number Diff line change
Expand Up @@ -1745,24 +1745,6 @@ var varGen = map[string]sessionVar{
},
},

`enable_experimental_stream_replication`: {
GetStringVal: makePostgresBoolGetStringValFn(`enable_experimental_stream_replication`),
Set: func(_ context.Context, m sessionDataMutator, s string) error {
b, err := paramparse.ParseBoolVar(`enable_experimental_stream_replication`, s)
if err != nil {
return err
}
m.SetStreamReplicationEnabled(b)
return nil
},
Get: func(evalCtx *extendedEvalContext, _ *kv.Txn) (string, error) {
return formatBoolAsPostgresSetting(evalCtx.SessionData().EnableStreamReplication), nil
},
GlobalDefault: func(sv *settings.Values) string {
return formatBoolAsPostgresSetting(experimentalStreamReplicationEnabled.Get(sv))
},
},

// CockroachDB extension. See experimentalComputedColumnRewrites or
// ParseComputedColumnRewrites for a description of the format.
`experimental_computed_column_rewrites`: {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -52,3 +52,14 @@ export const selectAutomaticStatsCollectionEnabled = createSelector(
return value === "true";
},
);

export const selectCrossClusterReplicationEnabled = createSelector(
selectClusterSettings,
(settings): boolean | undefined => {
if (!settings) {
return undefined;
}
const value = settings["cross_cluster_replication.enabled"]?.value;
return value === "true";
},
);
Loading

0 comments on commit bbc6ec4

Please sign in to comment.