From bbc6ec4f8deed8fbb6d95ef937e7291001dadbca Mon Sep 17 00:00:00 2001 From: adityamaru Date: Mon, 30 Jan 2023 12:04:57 -0500 Subject: [PATCH] server,db-console: add feature flag for Cross-Cluster Replication dashboard 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 --- .../settings/settings-for-tenants.txt | 3 --- docs/generated/settings/settings.html | 1 - .../replication_helpers.go | 2 +- .../replicationtestutils/testutils.go | 2 +- pkg/ccl/streamingccl/settings.go | 9 +++++++++ .../streamingest/alter_replication_job.go | 14 ++++++++++++++ .../streamingest/alter_replication_job_test.go | 1 + .../replication_random_client_test.go | 4 ++-- .../streamingest/stream_ingestion_job_test.go | 4 ++-- .../streamingest/stream_ingestion_planning.go | 9 ++++----- .../streamproducer/replication_stream_test.go | 2 +- pkg/cmd/roachtest/tests/cluster_to_cluster.go | 2 +- pkg/settings/registry.go | 4 ++-- pkg/sql/exec_util.go | 8 -------- .../testdata/logic_test/information_schema | 1 - .../logictest/testdata/logic_test/pg_catalog | 3 --- .../logictest/testdata/logic_test/show_source | 1 - pkg/sql/vars.go | 18 ------------------ .../clusterSettings.selectors.ts | 11 +++++++++++ .../cluster/containers/nodeGraphs/index.tsx | 17 ++++++++++++++--- 20 files changed, 63 insertions(+), 53 deletions(-) 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
enumerationoffdefault 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
booleanfalsedefault 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
booleanfalsedefault 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
booleanfalsedefault 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
booleanfalsedefault 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
integer10000default 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
duration0sdefault 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 = {