diff --git a/ci/gucs.out b/ci/gucs.out deleted file mode 100644 index ca989b5a0ee..00000000000 --- a/ci/gucs.out +++ /dev/null @@ -1,134 +0,0 @@ -"citus_tests_label_provider", -"citus.all_modifications_commutative", -"citus.allow_modifications_from_workers_to_replicated_tables", -"citus.allow_nested_distributed_execution", -"citus.allow_unsafe_constraints", -"citus.allow_unsafe_locks_from_workers", -"citus.background_task_queue_interval", -"citus.check_available_space_before_move", -"citus.cluster_name", -"citus.coordinator_aggregation_strategy", -"citus.copy_switchover_threshold", -"citus.count_distinct_error_rate", -"citus.cpu_priority", -"citus.cpu_priority_for_logical_replication_senders", -"citus.create_object_propagation", -"citus.defer_drop_after_shard_move", -"citus.defer_drop_after_shard_split", -"citus.defer_shard_delete_interval", -"citus.desired_percent_disk_available_after_move", -"citus.distributed_deadlock_detection_factor", -"citus.enable_alter_database_owner", -"citus.enable_alter_role_propagation", -"citus.enable_alter_role_set_propagation", -"citus.enable_binary_protocol", -"citus.enable_change_data_capture", -"citus.enable_cluster_clock", -"citus.enable_cost_based_connection_establishment", -"citus.enable_create_role_propagation", -"citus.enable_create_type_propagation", -"citus.enable_ddl_propagation", -"citus.enable_deadlock_prevention", -"citus.enable_fast_path_router_planner", -"citus.enable_local_execution", -"citus.enable_local_reference_table_foreign_keys", -"citus.enable_manual_changes_to_shards", -"citus.enable_manual_metadata_changes_for_user", -"citus.enable_metadata_sync", -"citus.enable_non_colocated_router_query_pushdown", -"citus.enable_repartition_joins", -"citus.enable_repartitioned_insert_select", -"citus.enable_router_execution", -"citus.enable_schema_based_sharding", -"citus.enable_single_hash_repartition_joins", -"citus.enable_statistics_collection", -"citus.enable_unique_job_ids", -"citus.enable_unsafe_triggers", -"citus.enable_unsupported_feature_messages", -"citus.enable_version_checks", -"citus.enforce_foreign_key_restrictions", -"citus.enforce_object_restrictions_for_local_objects", -"citus.executor_slow_start_interval", -"citus.explain_all_tasks", -"citus.explain_analyze_sort_method", -"citus.explain_distributed_queries", -"citus.force_max_query_parallelization", -"citus.function_opens_transaction_block", -"citus.grep_remote_commands", -"citus.hide_citus_dependent_objects", -"citus.hide_shards_from_app_name_prefixes", -"citus.isolation_test_session_process_id", -"citus.isolation_test_session_remote_process_id", -"citus.limit_clause_row_fetch_count", -"citus.local_copy_flush_threshold", -"citus.local_hostname", -"citus.local_shared_pool_size", -"citus.local_table_join_policy", -"citus.log_distributed_deadlock_detection", -"citus.log_intermediate_results", -"citus.log_local_commands", -"citus.log_multi_join_order", -"citus.log_remote_commands", -"citus.logical_replication_timeout", -"citus.main_db", -"citus.max_adaptive_executor_pool_size", -"citus.max_background_task_executors", -"citus.max_background_task_executors_per_node", -"citus.max_cached_connection_lifetime", -"citus.max_cached_conns_per_worker", -"citus.max_client_connections", -"citus.max_high_priority_background_processes", -"citus.max_intermediate_result_size", -"citus.max_matview_size_to_auto_recreate", -"citus.max_rebalancer_logged_ignored_moves", -"citus.max_shared_pool_size", -"citus.max_worker_nodes_tracked", -"citus.metadata_sync_interval", -"citus.metadata_sync_mode", -"citus.metadata_sync_retry_interval", -"citus.mitmfifo", -"citus.multi_shard_modify_mode", -"citus.multi_task_query_log_level", -"citus.next_cleanup_record_id", -"citus.next_operation_id", -"citus.next_placement_id", -"citus.next_shard_id", -"citus.node_connection_timeout", -"citus.node_conninfo", -"citus.override_table_visibility", -"citus.prevent_incomplete_connection_establishment", -"citus.propagate_session_settings_for_loopback_connection", -"citus.propagate_set_commands", -"citus.rebalancer_by_disk_size_base_cost", -"citus.recover_2pc_interval", -"citus.remote_copy_flush_threshold", -"citus.remote_task_check_interval", -"citus.repartition_join_bucket_count_per_node", -"citus.replicate_reference_tables_on_activate", -"citus.replication_model", -"citus.running_under_citus_test_suite", -"citus.select_opens_transaction_block", -"citus.shard_count", -"citus.shard_replication_factor", -"citus.show_shards_for_app_name_prefixes", -"citus.skip_advisory_lock_permission_checks", -"citus.skip_constraint_validation", -"citus.skip_jsonb_validation_in_copy", -"citus.sort_returning", -"citus.stat_statements_max", -"citus.stat_statements_purge_interval", -"citus.stat_statements_track", -"citus.stat_tenants_limit", -"citus.stat_tenants_log_level", -"citus.stat_tenants_period", -"citus.stat_tenants_track", -"citus.stat_tenants_untracked_sample_rate", -"citus.subquery_pushdown", -"citus.task_assignment_policy", -"citus.task_executor_type", -"citus.use_citus_managed_tables", -"citus.use_secondary_nodes", -"citus.values_materialization_threshold", -"citus.version", -"citus.worker_min_messages", -"citus.writable_standby_coordinator", diff --git a/src/backend/distributed/commands/distribute_object_ops.c b/src/backend/distributed/commands/distribute_object_ops.c index 663b8d04e36..72ea5beb42b 100644 --- a/src/backend/distributed/commands/distribute_object_ops.c +++ b/src/backend/distributed/commands/distribute_object_ops.c @@ -377,7 +377,7 @@ static DistributeObjectOps Any_Rename = { static DistributeObjectOps Any_SecLabel = { .deparse = DeparseSecLabelStmt, .qualify = NULL, - .preprocess = PreprocessSecLabelStmt, + .preprocess = NULL, .postprocess = PostprocessSecLabelStmt, .operationType = DIST_OPS_ALTER, .address = SecLabelStmtObjectAddress, diff --git a/src/backend/distributed/commands/role.c b/src/backend/distributed/commands/role.c index b89f34f55a1..1daf730e174 100644 --- a/src/backend/distributed/commands/role.c +++ b/src/backend/distributed/commands/role.c @@ -937,7 +937,7 @@ GenerateSecLabelOnRoleStmts(Oid roleid, char *rolename) { SecLabelStmt *secLabelStmt = makeNode(SecLabelStmt); secLabelStmt->objtype = OBJECT_ROLE; - secLabelStmt->object = (Node *) makeString(rolename); + secLabelStmt->object = (Node *) makeString(pstrdup(rolename)); Datum datumArray[Natts_pg_shseclabel]; bool isNullArray[Natts_pg_shseclabel]; diff --git a/src/backend/distributed/commands/seclabel.c b/src/backend/distributed/commands/seclabel.c index f3e12ea6e02..015f1cf63ea 100644 --- a/src/backend/distributed/commands/seclabel.c +++ b/src/backend/distributed/commands/seclabel.c @@ -19,18 +19,17 @@ #include "distributed/metadata_sync.h" #include "distributed/metadata/distobject.h" + /* - * PreprocessSecLabelStmt is executed before the statement is applied to the local - * postgres instance. - * - * In this stage we can prepare the commands that need to be run on all workers to assign + * PostprocessSecLabelStmt prepares the commands that need to be run on all workers to assign * security labels on distributed objects, currently supporting just Role objects. + * It also ensures that all object dependencies exist on all + * nodes for the object in the SecLabelStmt. */ List * -PreprocessSecLabelStmt(Node *node, const char *queryString, - ProcessUtilityContext processUtilityContext) +PostprocessSecLabelStmt(Node *node, const char *queryString) { - if (!IsCoordinator() || !ShouldPropagate()) + if (!ShouldPropagate()) { return NIL; } @@ -45,13 +44,17 @@ PreprocessSecLabelStmt(Node *node, const char *queryString, if (secLabelStmt->objtype != OBJECT_ROLE) { - if (EnableUnsupportedFeatureMessages) + /* + * If we are not in the coordinator, we don't want to interrupt the security + * label command with notices, the user expects that from the worker node + * the command will not be propagated + */ + if (EnableUnsupportedFeatureMessages && IsCoordinator()) { ereport(NOTICE, (errmsg("not propagating SECURITY LABEL commands whose " "object type is not role"), errhint("Connect to worker nodes directly to manually " - "run the same SECURITY LABEL command after " - "disabling DDL propagation."))); + "run the same SECURITY LABEL command."))); } return NIL; } @@ -61,6 +64,9 @@ PreprocessSecLabelStmt(Node *node, const char *queryString, return NIL; } + EnsureCoordinator(); + EnsureAllObjectDependenciesExistOnAllNodes(objectAddresses); + const char *sql = DeparseTreeNode((Node *) secLabelStmt); List *commandList = list_make3(DISABLE_DDL_PROPAGATION, @@ -71,36 +77,6 @@ PreprocessSecLabelStmt(Node *node, const char *queryString, } -/* - * PostprocessSecLabelStmt ensures that all object dependencies exist on all - * nodes for the object in the SecLabelStmt. Currently, we only support SecLabelStmts - * operating on a ROLE object. - */ -List * -PostprocessSecLabelStmt(Node *node, const char *queryString) -{ - if (!EnableCreateRolePropagation || !IsCoordinator() || !ShouldPropagate()) - { - return NIL; - } - - SecLabelStmt *secLabelStmt = castNode(SecLabelStmt, node); - - if (secLabelStmt->objtype != OBJECT_ROLE) - { - return NIL; - } - - List *objectAddresses = GetObjectAddressListFromParseTree(node, false, false); - if (IsAnyObjectDistributed(objectAddresses)) - { - EnsureAllObjectDependenciesExistOnAllNodes(objectAddresses); - } - - return NIL; -} - - /* * SecLabelStmtObjectAddress returns the object address of the object on * which this statement operates (secLabelStmt->object). Note that it has no limitation diff --git a/src/backend/distributed/deparser/deparse_seclabel_stmts.c b/src/backend/distributed/deparser/deparse_seclabel_stmts.c index ebd0736a289..a1aa047ccb9 100644 --- a/src/backend/distributed/deparser/deparse_seclabel_stmts.c +++ b/src/backend/distributed/deparser/deparse_seclabel_stmts.c @@ -44,7 +44,7 @@ AppendSecLabelStmt(StringInfo buf, SecLabelStmt *stmt) if (stmt->provider != NULL) { - appendStringInfo(buf, "FOR %s ", stmt->provider); + appendStringInfo(buf, "FOR %s ", quote_identifier(stmt->provider)); } appendStringInfoString(buf, "ON "); @@ -53,7 +53,7 @@ AppendSecLabelStmt(StringInfo buf, SecLabelStmt *stmt) { case OBJECT_ROLE: { - appendStringInfo(buf, "ROLE %s ", strVal(stmt->object)); + appendStringInfo(buf, "ROLE %s ", quote_identifier(strVal(stmt->object))); break; } diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 5b5ae87da6f..df98e66a5e2 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -522,8 +522,6 @@ extern List * AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok, bo isPostprocess); /* seclabel.c - forward declarations*/ -extern List * PreprocessSecLabelStmt(Node *node, const char *queryString, - ProcessUtilityContext processUtilityContext); extern List * PostprocessSecLabelStmt(Node *node, const char *queryString); extern List * SecLabelStmtObjectAddress(Node *node, bool missing_ok, bool isPostprocess); extern void citus_test_object_relabel(const ObjectAddress *object, const char *seclabel); diff --git a/src/test/regress/Makefile b/src/test/regress/Makefile index fe4868aa67a..4bdc7a1b8a5 100644 --- a/src/test/regress/Makefile +++ b/src/test/regress/Makefile @@ -101,7 +101,7 @@ check-base: all # check-minimal only sets up the cluster check-minimal: all - $(pg_regress_multi_check) --load-extension=citus --seclabel \ + $(pg_regress_multi_check) --load-extension=citus \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/minimal_schedule $(EXTRA_TESTS) check-base-vg: all @@ -118,7 +118,7 @@ check-minimal-mx: all -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/mx_minimal_schedule $(EXTRA_TESTS) check-custom-schedule: all - $(pg_regress_multi_check) --load-extension=citus --seclabel-test --worker-count=$(WORKERCOUNT) \ + $(pg_regress_multi_check) --load-extension=citus --worker-count=$(WORKERCOUNT) \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/$(SCHEDULE) $(EXTRA_TESTS) check-failure-custom-schedule: all @@ -159,7 +159,7 @@ check-enterprise: all $(pg_regress_multi_check) --load-extension=citus \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/enterprise_schedule $(EXTRA_TESTS) check-multi-1: all - $(pg_regress_multi_check) --load-extension=citus --seclabel-test \ + $(pg_regress_multi_check) --load-extension=citus \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_1_schedule $(EXTRA_TESTS) check-multi-hyperscale: all diff --git a/src/test/regress/expected/seclabel.out b/src/test/regress/expected/seclabel.out index 3cf2949b647..d5729f47352 100644 --- a/src/test/regress/expected/seclabel.out +++ b/src/test/regress/expected/seclabel.out @@ -11,14 +11,16 @@ SELECT citus_remove_node('localhost', :worker_2_port); (1 row) +-- create two roles, one with characters that need escaping CREATE ROLE user1; +CREATE ROLE "user 2"; -- check an invalid label for our current dummy hook citus_test_object_relabel SECURITY LABEL ON ROLE user1 IS 'invalid_label'; ERROR: 'invalid_label' is not a valid security label for Citus tests. -- if we disable metadata_sync, the command will not be propagated SET citus.enable_metadata_sync TO off; SECURITY LABEL ON ROLE user1 IS 'citus_unclassified'; -SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); +SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type; node_type | result --------------------------------------------------------------------- coordinator | {"label": "citus_unclassified", "objtype": "role"} @@ -28,6 +30,7 @@ SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); RESET citus.enable_metadata_sync; -- check that we only support propagating for roles SET citus.shard_replication_factor to 1; +-- distributed table CREATE TABLE a (a int); SELECT create_distributed_table('a', 'a'); create_distributed_table @@ -35,21 +38,73 @@ SELECT create_distributed_table('a', 'a'); (1 row) +-- distributed view +CREATE VIEW v_dist AS SELECT * FROM a; +-- distributed function +CREATE FUNCTION notice(text) RETURNS void LANGUAGE plpgsql AS $$ + BEGIN RAISE NOTICE '%', $1; END; $$; SECURITY LABEL ON TABLE a IS 'citus_classified'; NOTICE: not propagating SECURITY LABEL commands whose object type is not role -HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command after disabling DDL propagation. -SELECT node_type, result FROM get_citus_tests_label_provider_labels('a'); +HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command. +SECURITY LABEL ON FUNCTION notice IS 'citus_unclassified'; +NOTICE: not propagating SECURITY LABEL commands whose object type is not role +HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command. +SECURITY LABEL ON VIEW v_dist IS 'citus_classified'; +NOTICE: not propagating SECURITY LABEL commands whose object type is not role +HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command. +SELECT node_type, result FROM get_citus_tests_label_provider_labels('a') ORDER BY node_type; node_type | result --------------------------------------------------------------------- coordinator | {"label": "citus_classified", "objtype": "table"} worker_1 | (2 rows) -DROP TABLE a; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('notice(text)') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_unclassified", "objtype": "function"} + worker_1 | +(2 rows) + +SELECT node_type, result FROM get_citus_tests_label_provider_labels('v_dist') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_classified", "objtype": "view"} + worker_1 | +(2 rows) + +\c - - - :worker_1_port +SECURITY LABEL ON TABLE a IS 'citus_classified'; +SECURITY LABEL ON FUNCTION notice IS 'citus_unclassified'; +SECURITY LABEL ON VIEW v_dist IS 'citus_classified'; +\c - - - :master_port +SELECT node_type, result FROM get_citus_tests_label_provider_labels('a') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_classified", "objtype": "table"} + worker_1 | {"label": "citus_classified", "objtype": "table"} +(2 rows) + +SELECT node_type, result FROM get_citus_tests_label_provider_labels('notice(text)') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_unclassified", "objtype": "function"} + worker_1 | {"label": "citus_unclassified", "objtype": "function"} +(2 rows) + +SELECT node_type, result FROM get_citus_tests_label_provider_labels('v_dist') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_classified", "objtype": "view"} + worker_1 | {"label": "citus_classified", "objtype": "view"} +(2 rows) + +DROP TABLE a CASCADE; +NOTICE: drop cascades to view v_dist +DROP FUNCTION notice; +-- test that SECURITY LABEL statement is actually propagated for ROLES SET citus.log_remote_commands TO on; SET citus.grep_remote_commands = '%SECURITY LABEL%'; --- then we run a security label statement which will use the same connection to the worker node --- it should finish successfully SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_classified'; NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus_classified' DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx @@ -59,28 +114,44 @@ DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified'; NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified' DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +SECURITY LABEL for citus_tests_label_provider ON ROLE "user 2" IS 'citus_classified'; +NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE "user 2" IS 'citus_classified' +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +\c - - - :worker_1_port +-- command not allowed from worker node +SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified'; +ERROR: operation is not allowed on this node +HINT: Connect to the coordinator and run it again. +\c - - - :master_port RESET citus.log_remote_commands; -SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); +SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type; node_type | result --------------------------------------------------------------------- coordinator | {"label": "citus_unclassified", "objtype": "role"} worker_1 | {"label": "citus_unclassified", "objtype": "role"} (2 rows) --- adding a new node will fail because the label provider is not there --- however, this is enough for testing as we can see that the SECURITY LABEL commands --- will be propagated when adding a new node +SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_classified", "objtype": "role"} + worker_1 | {"label": "citus_classified", "objtype": "role"} +(2 rows) + +-- add a new node and check that it also propagates the SECURITY LABEL statement to the new node SET citus.log_remote_commands TO on; SET citus.grep_remote_commands = '%SECURITY LABEL%'; SELECT 1 FROM citus_add_node('localhost', :worker_2_port); NOTICE: issuing SELECT worker_create_or_alter_role('user1', 'CREATE ROLE user1 NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''', 'ALTER ROLE user1 NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''');SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified' +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +NOTICE: issuing SELECT worker_create_or_alter_role('user 2', 'CREATE ROLE "user 2" NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''', 'ALTER ROLE "user 2" NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''');SECURITY LABEL FOR citus_tests_label_provider ON ROLE "user 2" IS 'citus_classified' DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx ?column? --------------------------------------------------------------------- 1 (1 row) -SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); +SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type; node_type | result --------------------------------------------------------------------- coordinator | {"label": "citus_unclassified", "objtype": "role"} @@ -88,6 +159,14 @@ SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); worker_2 | {"label": "citus_unclassified", "objtype": "role"} (3 rows) +SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_classified", "objtype": "role"} + worker_1 | {"label": "citus_classified", "objtype": "role"} + worker_2 | {"label": "citus_classified", "objtype": "role"} +(3 rows) + -- cleanup RESET citus.log_remote_commands; -DROP ROLE user1; +DROP ROLE user1, "user 2"; diff --git a/src/test/regress/pg_regress_multi.pl b/src/test/regress/pg_regress_multi.pl index 4def6dc7982..804cff20550 100755 --- a/src/test/regress/pg_regress_multi.pl +++ b/src/test/regress/pg_regress_multi.pl @@ -50,7 +50,6 @@ () print " --connection-timeout Timeout for connecting to worker nodes\n"; print " --mitmproxy Start a mitmproxy for one of the workers\n"; print " --worker-count Number of workers in Citus cluster (default: 2)\n"; - print " --seclabel-test This test runs seclabel propagation tests"; exit 1; } @@ -87,7 +86,6 @@ () my $publicWorker1Host = "localhost"; my $publicWorker2Host = "localhost"; my $workerCount = 2; -my $seclabelTest = 0; my $serversAreShutdown = "TRUE"; my $usingWindows = 0; @@ -122,7 +120,6 @@ () 'worker-1-public-hostname=s' => \$publicWorker1Host, 'worker-2-public-hostname=s' => \$publicWorker2Host, 'worker-count=i' => \$workerCount, - 'seclabel-test' => \$seclabelTest, 'help' => sub { Usage() }); my $fixopen = "$bindir/postgres.fixopen"; @@ -495,6 +492,9 @@ sub generate_hba push(@pgOptions, "citus.stat_tenants_limit = 2"); push(@pgOptions, "citus.stat_tenants_track = 'ALL'"); +# We currently need this config for isolation tests and security label tests +push(@pgOptions, "citus.running_under_citus_test_suite=true"); + # Some tests look at shards in pg_class, make sure we can usually see them: push(@pgOptions, "citus.show_shards_for_app_name_prefixes='pg_regress'"); @@ -563,7 +563,6 @@ sub generate_hba push(@pgOptions, "citus.metadata_sync_interval=1000"); push(@pgOptions, "citus.metadata_sync_retry_interval=100"); push(@pgOptions, "client_min_messages='warning'"); # pg12 introduced notice showing during isolation tests - push(@pgOptions, "citus.running_under_citus_test_suite=true"); # Disable all features of the maintenance daemon. Otherwise queries might # randomly show temporarily as "waiting..." because they are waiting for the @@ -576,14 +575,6 @@ sub generate_hba push(@pgOptions, "citus.background_task_queue_interval=-1"); } -# if the security label propagation tests are running in the suite -# we need to load the label provider in PG_init by setting -# running_under_citus_test_suite GUC to true -if($seclabelTest) -{ - push(@pgOptions, "citus.running_under_citus_test_suite=true"); -} - # Add externally added options last, so they overwrite the default ones above for my $option (@userPgOptions) { diff --git a/src/test/regress/sql/seclabel.sql b/src/test/regress/sql/seclabel.sql index b6f7980f245..c081721e239 100644 --- a/src/test/regress/sql/seclabel.sql +++ b/src/test/regress/sql/seclabel.sql @@ -8,7 +8,9 @@ -- citus_add_node later SELECT citus_remove_node('localhost', :worker_2_port); +-- create two roles, one with characters that need escaping CREATE ROLE user1; +CREATE ROLE "user 2"; -- check an invalid label for our current dummy hook citus_test_object_relabel SECURITY LABEL ON ROLE user1 IS 'invalid_label'; @@ -16,40 +18,69 @@ SECURITY LABEL ON ROLE user1 IS 'invalid_label'; -- if we disable metadata_sync, the command will not be propagated SET citus.enable_metadata_sync TO off; SECURITY LABEL ON ROLE user1 IS 'citus_unclassified'; -SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); +SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type; RESET citus.enable_metadata_sync; -- check that we only support propagating for roles SET citus.shard_replication_factor to 1; +-- distributed table CREATE TABLE a (a int); SELECT create_distributed_table('a', 'a'); +-- distributed view +CREATE VIEW v_dist AS SELECT * FROM a; +-- distributed function +CREATE FUNCTION notice(text) RETURNS void LANGUAGE plpgsql AS $$ + BEGIN RAISE NOTICE '%', $1; END; $$; + +SECURITY LABEL ON TABLE a IS 'citus_classified'; +SECURITY LABEL ON FUNCTION notice IS 'citus_unclassified'; +SECURITY LABEL ON VIEW v_dist IS 'citus_classified'; + +SELECT node_type, result FROM get_citus_tests_label_provider_labels('a') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('notice(text)') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('v_dist') ORDER BY node_type; + +\c - - - :worker_1_port SECURITY LABEL ON TABLE a IS 'citus_classified'; -SELECT node_type, result FROM get_citus_tests_label_provider_labels('a'); -DROP TABLE a; +SECURITY LABEL ON FUNCTION notice IS 'citus_unclassified'; +SECURITY LABEL ON VIEW v_dist IS 'citus_classified'; +\c - - - :master_port +SELECT node_type, result FROM get_citus_tests_label_provider_labels('a') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('notice(text)') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('v_dist') ORDER BY node_type; + +DROP TABLE a CASCADE; +DROP FUNCTION notice; + +-- test that SECURITY LABEL statement is actually propagated for ROLES SET citus.log_remote_commands TO on; SET citus.grep_remote_commands = '%SECURITY LABEL%'; --- then we run a security label statement which will use the same connection to the worker node --- it should finish successfully SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_classified'; SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS NULL; SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified'; +SECURITY LABEL for citus_tests_label_provider ON ROLE "user 2" IS 'citus_classified'; + +\c - - - :worker_1_port +-- command not allowed from worker node +SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified'; +\c - - - :master_port RESET citus.log_remote_commands; -SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); +SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type; --- adding a new node will fail because the label provider is not there --- however, this is enough for testing as we can see that the SECURITY LABEL commands --- will be propagated when adding a new node +-- add a new node and check that it also propagates the SECURITY LABEL statement to the new node SET citus.log_remote_commands TO on; SET citus.grep_remote_commands = '%SECURITY LABEL%'; SELECT 1 FROM citus_add_node('localhost', :worker_2_port); -SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1'); +SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type; -- cleanup RESET citus.log_remote_commands; -DROP ROLE user1; +DROP ROLE user1, "user 2";