diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt
index aa505b41e955..4248fcea0444 100644
--- a/docs/generated/settings/settings-for-tenants.txt
+++ b/docs/generated/settings/settings-for-tenants.txt
@@ -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"
diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html
index 62825f595d8a..11081ead30f9 100644
--- a/docs/generated/settings/settings.html
+++ b/docs/generated/settings/settings.html
@@ -135,7 +135,6 @@
sql.defaults.experimental_distsql_planning
| enumeration | off | default experimental_distsql_planning mode; enables experimental opt-driven DistSQL planning [off = 0, on = 1] This cluster setting is being kept to preserve backwards-compatibility. This session variable default should now be configured using ALTER ROLE... SET |
sql.defaults.experimental_enable_unique_without_index_constraints.enabled
| boolean | false | default value for experimental_enable_unique_without_index_constraints session setting;disables unique without index constraints by default This cluster setting is being kept to preserve backwards-compatibility. 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 |
-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 |
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 |
sql.defaults.foreign_key_cascades_limit
| integer | 10000 | default value for foreign_key_cascades_limit session setting; limits the number of cascading operations that run as part of a single query This cluster setting is being kept to preserve backwards-compatibility. This session variable default should now be configured using ALTER ROLE... SET |
sql.defaults.idle_in_session_timeout
| duration | 0s | 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 This cluster setting is being kept to preserve backwards-compatibility. This session variable default should now be configured using ALTER ROLE... SET |
diff --git a/pkg/ccl/streamingccl/replicationtestutils/replication_helpers.go b/pkg/ccl/streamingccl/replicationtestutils/replication_helpers.go
index 0b33bd078476..80c6fe418837 100644
--- a/pkg/ccl/streamingccl/replicationtestutils/replication_helpers.go
+++ b/pkg/ccl/streamingccl/replicationtestutils/replication_helpers.go
@@ -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.
diff --git a/pkg/ccl/streamingccl/replicationtestutils/testutils.go b/pkg/ccl/streamingccl/replicationtestutils/testutils.go
index 9d370d63d04a..847bb513cbe5 100644
--- a/pkg/ccl/streamingccl/replicationtestutils/testutils.go
+++ b/pkg/ccl/streamingccl/replicationtestutils/testutils.go
@@ -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()
diff --git a/pkg/ccl/streamingccl/settings.go b/pkg/ccl/streamingccl/settings.go
index ebaff1db04f4..cdcc62ee44b6 100644
--- a/pkg/ccl/streamingccl/settings.go
+++ b/pkg/ccl/streamingccl/settings.go
@@ -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(
diff --git a/pkg/ccl/streamingccl/streamingest/alter_replication_job.go b/pkg/ccl/streamingccl/streamingest/alter_replication_job.go
index faa9562ef314..4644ef86afbe 100644
--- a/pkg/ccl/streamingccl/streamingest/alter_replication_job.go
+++ b/pkg/ccl/streamingccl/streamingest/alter_replication_job.go
@@ -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"
@@ -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
}
diff --git a/pkg/ccl/streamingccl/streamingest/alter_replication_job_test.go b/pkg/ccl/streamingccl/streamingest/alter_replication_job_test.go
index ec17cfb30764..cc1424251297 100644
--- a/pkg/ccl/streamingccl/streamingest/alter_replication_job_test.go
+++ b/pkg/ccl/streamingccl/streamingest/alter_replication_job_test.go
@@ -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")
})
diff --git a/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go b/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go
index def502dec46f..9fd3f7041cba 100644
--- a/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go
+++ b/pkg/ccl/streamingccl/streamingest/replication_random_client_test.go
@@ -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)
diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go
index 4ff05a3f5a70..5dfc1872461b 100644
--- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go
+++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_job_test.go
@@ -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;
`,
";")...)
@@ -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))
diff --git a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go
index 249533858a62..9f77c1dd61cc 100644
--- a/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go
+++ b/pkg/ccl/streamingccl/streamingest/stream_ingestion_planning.go
@@ -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",
)
}
diff --git a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go
index 8314de7806f8..c416b85911b9 100644
--- a/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go
+++ b/pkg/ccl/streamingccl/streamproducer/replication_stream_test.go
@@ -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()
diff --git a/pkg/cmd/roachtest/tests/cluster_to_cluster.go b/pkg/cmd/roachtest/tests/cluster_to_cluster.go
index f42aff2ac8cd..dcf348b5dac6 100644
--- a/pkg/cmd/roachtest/tests/cluster_to_cluster.go
+++ b/pkg/cmd/roachtest/tests/cluster_to_cluster.go
@@ -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(
diff --git a/pkg/settings/registry.go b/pkg/settings/registry.go
index efbbb3e4e4fd..05844f866d32 100644
--- a/pkg/settings/registry.go
+++ b/pkg/settings/registry.go
@@ -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
@@ -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": {},
diff --git a/pkg/sql/exec_util.go b/pkg/sql/exec_util.go
index 15f6f9776713..d1f991066f0c 100644
--- a/pkg/sql/exec_util.go
+++ b/pkg/sql/exec_util.go
@@ -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`,
diff --git a/pkg/sql/logictest/testdata/logic_test/information_schema b/pkg/sql/logictest/testdata/logic_test/information_schema
index d1acf82d6264..091ca9f0f05f 100644
--- a/pkg/sql/logictest/testdata/logic_test/information_schema
+++ b/pkg/sql/logictest/testdata/logic_test/information_schema
@@ -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
diff --git a/pkg/sql/logictest/testdata/logic_test/pg_catalog b/pkg/sql/logictest/testdata/logic_test/pg_catalog
index ee94dd9d7550..fe282b88117e 100644
--- a/pkg/sql/logictest/testdata/logic_test/pg_catalog
+++ b/pkg/sql/logictest/testdata/logic_test/pg_catalog
@@ -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
@@ -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
@@ -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
diff --git a/pkg/sql/logictest/testdata/logic_test/show_source b/pkg/sql/logictest/testdata/logic_test/show_source
index d766199fe25a..0241725c4dba 100644
--- a/pkg/sql/logictest/testdata/logic_test/show_source
+++ b/pkg/sql/logictest/testdata/logic_test/show_source
@@ -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
diff --git a/pkg/sql/vars.go b/pkg/sql/vars.go
index f0648b65938b..b98e10818fb0 100644
--- a/pkg/sql/vars.go
+++ b/pkg/sql/vars.go
@@ -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`: {
diff --git a/pkg/ui/workspaces/db-console/src/redux/clusterSettings/clusterSettings.selectors.ts b/pkg/ui/workspaces/db-console/src/redux/clusterSettings/clusterSettings.selectors.ts
index 62d0e3601e27..efb446a5bed4 100644
--- a/pkg/ui/workspaces/db-console/src/redux/clusterSettings/clusterSettings.selectors.ts
+++ b/pkg/ui/workspaces/db-console/src/redux/clusterSettings/clusterSettings.selectors.ts
@@ -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";
+ },
+);
diff --git a/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeGraphs/index.tsx b/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeGraphs/index.tsx
index 5fbc38cea82d..81bd8711b389 100644
--- a/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeGraphs/index.tsx
+++ b/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeGraphs/index.tsx
@@ -64,7 +64,7 @@ import hardwareDashboard from "./dashboards/hardware";
import changefeedsDashboard from "./dashboards/changefeeds";
import overloadDashboard from "./dashboards/overload";
import ttlDashboard from "./dashboards/ttl";
-import crossClusterReplication from "./dashboards/crossClusterReplication";
+import crossClusterReplicationDashboard from "./dashboards/crossClusterReplication";
import { getMatchParamByName } from "src/util/query";
import { PayloadAction } from "src/interfaces/action";
import {
@@ -85,7 +85,9 @@ import moment from "moment";
import {
selectResolution10sStorageTTL,
selectResolution30mStorageTTL,
+ selectCrossClusterReplicationEnabled,
} from "src/redux/clusterSettings";
+
interface GraphDashboard {
label: string;
component: (props: GraphDashboardProps) => React.ReactElement[];
@@ -106,7 +108,7 @@ const dashboards: { [key: string]: GraphDashboard } = {
ttl: { label: "TTL", component: ttlDashboard },
crossClusterReplication: {
label: "Cross-Cluster Replication",
- component: crossClusterReplication,
+ component: crossClusterReplicationDashboard,
},
};
@@ -132,6 +134,7 @@ type MapStateToProps = {
nodeDisplayNameByID: ReturnType<
typeof nodeDisplayNameByIDSelector.resultFunc
>;
+ crossClusterReplicationEnabled: boolean;
};
type MapDispatchToProps = {
@@ -171,6 +174,7 @@ export class NodeGraphs extends React.Component<
refresh = () => {
this.props.refreshNodes();
this.props.refreshLiveness();
+ this.props.refreshNodeSettings();
};
setClusterPath(nodeID: string, dashboardName: string) {
@@ -322,6 +326,12 @@ export class NodeGraphs extends React.Component<
const paddingBottom =
nodeIDs.length > 8 ? 90 + Math.ceil(nodeIDs.length / 3) * 10 : 50;
+ const filteredDropdownOptions = this.props.crossClusterReplicationEnabled
+ ? dashboardDropdownOptions // Already in the list, no need to filter
+ : dashboardDropdownOptions.filter(
+ option => option.label !== "Cross-Cluster Replication",
+ );
+
return (
@@ -338,7 +348,7 @@ export class NodeGraphs extends React.Component<
({
storeIDsByNodeID: selectStoreIDsByNodeID(state),
nodeDropdownOptions: nodeDropdownOptionsSelector(state),
nodeDisplayNameByID: nodeDisplayNameByIDSelector(state),
+ crossClusterReplicationEnabled: selectCrossClusterReplicationEnabled(state),
});
const mapDispatchToProps: MapDispatchToProps = {