From 1468ae33674da892fe32cdd4bec263e55f36f071 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Emel=20=C5=9Eim=C5=9Fek?= Date: Wed, 13 Nov 2024 14:21:08 +0300 Subject: [PATCH] Propagates SECURITY LABEL ON ROLE stmt (#7304) (#7735) Propagates SECURITY LABEL ON ROLE stmt (https://github.com/citusdata/citus/pull/7304) We propagate `SECURITY LABEL [for provider] ON ROLE rolename IS labelname` to the worker nodes. We also make sure to run the relevant `SecLabelStmt` commands on a newly added node by looking at roles found in `pg_shseclabel`. See official docs for explanation on how this command works: https://www.postgresql.org/docs/current/sql-security-label.html This command stores the role label in the `pg_shseclabel` catalog table. This commit also fixes the regex string in `check_gucs_are_alphabetically_sorted.sh` script such that it escapes the dot. Previously it was looking for all strings starting with "citus" instead of "citus." as it should. To test this feature, I currently make use of a special GUC to control label provider registration in PG_init when creating the Citus extension. (cherry picked from commit 0d1f18862be68350a2f784bcbe98344ddcad8a6d) Co-authored-by: Naisila Puka <37271756+naisila@users.noreply.github.com> (cherry picked from commit 686d2b46cadf8f8257797506c677d24c5598ba52) --- ci/check_gucs_are_alphabetically_sorted.sh | 2 +- gucs.out | 133 ++++++++++++++ .../commands/distribute_object_ops.c | 14 ++ src/backend/distributed/commands/role.c | 71 ++++++- src/backend/distributed/commands/seclabel.c | 125 +++++++++++++ .../deparser/deparse_seclabel_stmts.c | 79 ++++++++ .../distributed/operations/shard_rebalancer.c | 2 +- .../replication/multi_logical_replication.c | 4 +- src/backend/distributed/shared_library_init.c | 18 +- src/include/distributed/commands.h | 5 + src/include/distributed/deparser.h | 3 + src/include/distributed/shard_rebalancer.h | 2 +- .../regress/expected/multi_test_helpers.out | 30 +++ src/test/regress/expected/seclabel.out | 173 ++++++++++++++++++ src/test/regress/multi_1_schedule | 1 + src/test/regress/pg_regress_multi.pl | 7 +- src/test/regress/sql/multi_test_helpers.sql | 30 +++ src/test/regress/sql/seclabel.sql | 87 +++++++++ 18 files changed, 774 insertions(+), 12 deletions(-) create mode 100644 gucs.out create mode 100644 src/backend/distributed/commands/seclabel.c create mode 100644 src/backend/distributed/deparser/deparse_seclabel_stmts.c create mode 100644 src/test/regress/expected/seclabel.out create mode 100644 src/test/regress/sql/seclabel.sql diff --git a/ci/check_gucs_are_alphabetically_sorted.sh b/ci/check_gucs_are_alphabetically_sorted.sh index a769ae4fb16..763b5305f81 100755 --- a/ci/check_gucs_are_alphabetically_sorted.sh +++ b/ci/check_gucs_are_alphabetically_sorted.sh @@ -5,6 +5,6 @@ set -euo pipefail source ci/ci_helpers.sh # extract citus gucs in the form of "citus.X" -grep -o -E "(\.*\"citus.\w+\")," src/backend/distributed/shared_library_init.c > gucs.out +grep -o -E "(\.*\"citus\.\w+\")," src/backend/distributed/shared_library_init.c > gucs.out sort -c gucs.out rm gucs.out diff --git a/gucs.out b/gucs.out new file mode 100644 index 00000000000..8501e6c1f75 --- /dev/null +++ b/gucs.out @@ -0,0 +1,133 @@ +"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 16f6507df93..0b476e496ff 100644 --- a/src/backend/distributed/commands/distribute_object_ops.c +++ b/src/backend/distributed/commands/distribute_object_ops.c @@ -364,6 +364,15 @@ static DistributeObjectOps Any_Rename = { .address = NULL, .markDistributed = false, }; +static DistributeObjectOps Any_SecLabel = { + .deparse = DeparseSecLabelStmt, + .qualify = NULL, + .preprocess = NULL, + .postprocess = PostprocessSecLabelStmt, + .operationType = DIST_OPS_ALTER, + .address = SecLabelStmtObjectAddress, + .markDistributed = false, +}; static DistributeObjectOps Attribute_Rename = { .deparse = DeparseRenameAttributeStmt, .qualify = QualifyRenameAttributeStmt, @@ -1991,6 +2000,11 @@ GetDistributeObjectOps(Node *node) return &Vacuum_Analyze; } + case T_SecLabelStmt: + { + return &Any_SecLabel; + } + case T_RenameStmt: { RenameStmt *stmt = castNode(RenameStmt, node); diff --git a/src/backend/distributed/commands/role.c b/src/backend/distributed/commands/role.c index 20853901850..6a42211a6ab 100644 --- a/src/backend/distributed/commands/role.c +++ b/src/backend/distributed/commands/role.c @@ -22,6 +22,7 @@ #include "catalog/pg_auth_members.h" #include "catalog/pg_authid.h" #include "catalog/pg_db_role_setting.h" +#include "catalog/pg_shseclabel.h" #include "catalog/pg_type.h" #include "commands/dbcommands.h" #include "nodes/makefuncs.h" @@ -65,6 +66,7 @@ static DefElem * makeDefElemBool(char *name, bool value); static List * GenerateRoleOptionsList(HeapTuple tuple); static List * GenerateGrantRoleStmtsFromOptions(RoleSpec *roleSpec, List *options); static List * GenerateGrantRoleStmtsOfRole(Oid roleid); +static List * GenerateSecLabelOnRoleStmts(Oid roleid, char *rolename); static void EnsureSequentialModeForRoleDDL(void); static char * GetRoleNameFromDbRoleSetting(HeapTuple tuple, @@ -516,13 +518,14 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid) { HeapTuple roleTuple = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleOid)); Form_pg_authid role = ((Form_pg_authid) GETSTRUCT(roleTuple)); + char *rolename = pstrdup(NameStr(role->rolname)); CreateRoleStmt *createRoleStmt = NULL; if (EnableCreateRolePropagation) { createRoleStmt = makeNode(CreateRoleStmt); createRoleStmt->stmt_type = ROLESTMT_ROLE; - createRoleStmt->role = pstrdup(NameStr(role->rolname)); + createRoleStmt->role = rolename; createRoleStmt->options = GenerateRoleOptionsList(roleTuple); } @@ -533,7 +536,7 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid) alterRoleStmt->role = makeNode(RoleSpec); alterRoleStmt->role->roletype = ROLESPEC_CSTRING; alterRoleStmt->role->location = -1; - alterRoleStmt->role->rolename = pstrdup(NameStr(role->rolname)); + alterRoleStmt->role->rolename = rolename; alterRoleStmt->action = 1; alterRoleStmt->options = GenerateRoleOptionsList(roleTuple); } @@ -545,7 +548,7 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid) { /* add a worker_create_or_alter_role command if any of them are set */ char *createOrAlterRoleQuery = CreateCreateOrAlterRoleCommand( - pstrdup(NameStr(role->rolname)), + rolename, createRoleStmt, alterRoleStmt); @@ -567,6 +570,20 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid) { completeRoleList = lappend(completeRoleList, DeparseTreeNode(stmt)); } + + /* + * append SECURITY LABEL ON ROLE commands for this specific user + * When we propagate user creation, we also want to make sure that we propagate + * all the security labels it has been given. For this, we check pg_shseclabel + * for the ROLE entry corresponding to roleOid, and generate the relevant + * SecLabel stmts to be run in the new node. + */ + List *secLabelOnRoleStmts = GenerateSecLabelOnRoleStmts(roleOid, rolename); + stmt = NULL; + foreach_declared_ptr(stmt, secLabelOnRoleStmts) + { + completeRoleList = lappend(completeRoleList, DeparseTreeNode(stmt)); + } } return completeRoleList; @@ -896,6 +913,54 @@ GenerateGrantRoleStmtsOfRole(Oid roleid) } +/* + * GenerateSecLabelOnRoleStmts generates the SecLabelStmts for the role + * whose oid is roleid. + */ +static List * +GenerateSecLabelOnRoleStmts(Oid roleid, char *rolename) +{ + List *secLabelStmts = NIL; + + /* + * Note that roles are shared database objects, therefore their + * security labels are stored in pg_shseclabel instead of pg_seclabel. + */ + Relation pg_shseclabel = table_open(SharedSecLabelRelationId, AccessShareLock); + ScanKeyData skey[1]; + ScanKeyInit(&skey[0], Anum_pg_shseclabel_objoid, BTEqualStrategyNumber, F_OIDEQ, + ObjectIdGetDatum(roleid)); + SysScanDesc scan = systable_beginscan(pg_shseclabel, SharedSecLabelObjectIndexId, + true, NULL, 1, &skey[0]); + + HeapTuple tuple = NULL; + while (HeapTupleIsValid(tuple = systable_getnext(scan))) + { + SecLabelStmt *secLabelStmt = makeNode(SecLabelStmt); + secLabelStmt->objtype = OBJECT_ROLE; + secLabelStmt->object = (Node *) makeString(pstrdup(rolename)); + + Datum datumArray[Natts_pg_shseclabel]; + bool isNullArray[Natts_pg_shseclabel]; + + heap_deform_tuple(tuple, RelationGetDescr(pg_shseclabel), datumArray, + isNullArray); + + secLabelStmt->provider = TextDatumGetCString( + datumArray[Anum_pg_shseclabel_provider - 1]); + secLabelStmt->label = TextDatumGetCString( + datumArray[Anum_pg_shseclabel_label - 1]); + + secLabelStmts = lappend(secLabelStmts, secLabelStmt); + } + + systable_endscan(scan); + table_close(pg_shseclabel, AccessShareLock); + + return secLabelStmts; +} + + /* * PreprocessCreateRoleStmt creates a worker_create_or_alter_role query for the * role that is being created. With that query we can create the role in the diff --git a/src/backend/distributed/commands/seclabel.c b/src/backend/distributed/commands/seclabel.c new file mode 100644 index 00000000000..3e1847dc9ee --- /dev/null +++ b/src/backend/distributed/commands/seclabel.c @@ -0,0 +1,125 @@ +/*------------------------------------------------------------------------- + * + * seclabel.c + * + * This file contains the logic of SECURITY LABEL statement propagation. + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "distributed/commands.h" +#include "distributed/commands/utility_hook.h" +#include "distributed/coordinator_protocol.h" +#include "distributed/deparser.h" +#include "distributed/log_utils.h" +#include "distributed/metadata/distobject.h" +#include "distributed/metadata_sync.h" + + +/* + * 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 * +PostprocessSecLabelStmt(Node *node, const char *queryString) +{ + if (!ShouldPropagate()) + { + return NIL; + } + + SecLabelStmt *secLabelStmt = castNode(SecLabelStmt, node); + + List *objectAddresses = GetObjectAddressListFromParseTree(node, false, true); + if (!IsAnyObjectDistributed(objectAddresses)) + { + return NIL; + } + + if (secLabelStmt->objtype != OBJECT_ROLE) + { + /* + * 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."))); + } + return NIL; + } + + if (!EnableCreateRolePropagation) + { + return NIL; + } + + EnsureCoordinator(); + EnsureAllObjectDependenciesExistOnAllNodes(objectAddresses); + + const char *sql = DeparseTreeNode((Node *) secLabelStmt); + + List *commandList = list_make3(DISABLE_DDL_PROPAGATION, + (void *) sql, + ENABLE_DDL_PROPAGATION); + + return NodeDDLTaskList(NON_COORDINATOR_NODES, commandList); +} + + +/* + * SecLabelStmtObjectAddress returns the object address of the object on + * which this statement operates (secLabelStmt->object). Note that it has no limitation + * on the object type being OBJECT_ROLE. This is intentionally implemented like this + * since it is fairly simple to implement and we might extend SECURITY LABEL propagation + * in the future to include more object types. + */ +List * +SecLabelStmtObjectAddress(Node *node, bool missing_ok, bool isPostprocess) +{ + SecLabelStmt *secLabelStmt = castNode(SecLabelStmt, node); + + Relation rel = NULL; + ObjectAddress address = get_object_address(secLabelStmt->objtype, + secLabelStmt->object, &rel, + AccessShareLock, missing_ok); + if (rel != NULL) + { + relation_close(rel, AccessShareLock); + } + + ObjectAddress *addressPtr = palloc0(sizeof(ObjectAddress)); + *addressPtr = address; + return list_make1(addressPtr); +} + + +/* + * citus_test_object_relabel is a dummy function for check_object_relabel_type hook. + * It is meant to be used in tests combined with citus_test_register_label_provider + */ +void +citus_test_object_relabel(const ObjectAddress *object, const char *seclabel) +{ + if (seclabel == NULL || + strcmp(seclabel, "citus_unclassified") == 0 || + strcmp(seclabel, "citus_classified") == 0 || + strcmp(seclabel, "citus '!unclassified") == 0) + { + return; + } + + ereport(ERROR, + (errcode(ERRCODE_INVALID_NAME), + errmsg("'%s' is not a valid security label for Citus tests.", seclabel))); +} diff --git a/src/backend/distributed/deparser/deparse_seclabel_stmts.c b/src/backend/distributed/deparser/deparse_seclabel_stmts.c new file mode 100644 index 00000000000..ffe775b76ee --- /dev/null +++ b/src/backend/distributed/deparser/deparse_seclabel_stmts.c @@ -0,0 +1,79 @@ +/*------------------------------------------------------------------------- + * + * deparse_seclabel_stmts.c + * All routines to deparse SECURITY LABEL statements. + * + * Copyright (c), Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" + +#include "nodes/parsenodes.h" +#include "utils/builtins.h" + +#include "distributed/deparser.h" + +static void AppendSecLabelStmt(StringInfo buf, SecLabelStmt *stmt); + +/* + * DeparseSecLabelStmt builds and returns a string representing of the + * SecLabelStmt for application on a remote server. + */ +char * +DeparseSecLabelStmt(Node *node) +{ + SecLabelStmt *secLabelStmt = castNode(SecLabelStmt, node); + StringInfoData buf = { 0 }; + initStringInfo(&buf); + + AppendSecLabelStmt(&buf, secLabelStmt); + + return buf.data; +} + + +/* + * AppendSecLabelStmt generates the string representation of the + * SecLabelStmt and appends it to the buffer. + */ +static void +AppendSecLabelStmt(StringInfo buf, SecLabelStmt *stmt) +{ + appendStringInfoString(buf, "SECURITY LABEL "); + + if (stmt->provider != NULL) + { + appendStringInfo(buf, "FOR %s ", quote_identifier(stmt->provider)); + } + + appendStringInfoString(buf, "ON "); + + switch (stmt->objtype) + { + case OBJECT_ROLE: + { + appendStringInfo(buf, "ROLE %s ", quote_identifier(strVal(stmt->object))); + break; + } + + /* normally, we shouldn't reach this */ + default: + { + ereport(ERROR, (errmsg("unsupported security label statement for" + " deparsing"))); + } + } + + appendStringInfoString(buf, "IS "); + + if (stmt->label != NULL) + { + appendStringInfo(buf, "%s", quote_literal_cstr(stmt->label)); + } + else + { + appendStringInfoString(buf, "NULL"); + } +} diff --git a/src/backend/distributed/operations/shard_rebalancer.c b/src/backend/distributed/operations/shard_rebalancer.c index a4c7364bee9..074f1bed00e 100644 --- a/src/backend/distributed/operations/shard_rebalancer.c +++ b/src/backend/distributed/operations/shard_rebalancer.c @@ -319,7 +319,7 @@ PG_FUNCTION_INFO_V1(citus_rebalance_start); PG_FUNCTION_INFO_V1(citus_rebalance_stop); PG_FUNCTION_INFO_V1(citus_rebalance_wait); -bool RunningUnderIsolationTest = false; +bool RunningUnderCitusTestSuite = false; int MaxRebalancerLoggedIgnoredMoves = 5; int RebalancerByDiskSizeBaseCost = 100 * 1024 * 1024; bool PropagateSessionSettingsForLoopbackConnection = false; diff --git a/src/backend/distributed/replication/multi_logical_replication.c b/src/backend/distributed/replication/multi_logical_replication.c index 2dbf8daa805..9f8bb6a547d 100644 --- a/src/backend/distributed/replication/multi_logical_replication.c +++ b/src/backend/distributed/replication/multi_logical_replication.c @@ -1143,7 +1143,7 @@ ConflictWithIsolationTestingBeforeCopy(void) const bool sessionLock = false; const bool dontWait = false; - if (RunningUnderIsolationTest) + if (RunningUnderCitusTestSuite) { SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, SHARD_MOVE_ADVISORY_LOCK_SECOND_KEY, @@ -1177,7 +1177,7 @@ ConflictWithIsolationTestingAfterCopy(void) const bool sessionLock = false; const bool dontWait = false; - if (RunningUnderIsolationTest) + if (RunningUnderCitusTestSuite) { SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, SHARD_MOVE_ADVISORY_LOCK_FIRST_KEY, diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 21ae6b4b5b0..d7fa094ed75 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -27,6 +27,7 @@ #include "catalog/pg_extension.h" #include "commands/explain.h" #include "commands/extension.h" +#include "commands/seclabel.h" #include "common/string.h" #include "executor/executor.h" #include "libpq/auth.h" @@ -572,6 +573,16 @@ _PG_init(void) INIT_COLUMNAR_SYMBOL(PGFunction, columnar_storage_info); INIT_COLUMNAR_SYMBOL(PGFunction, columnar_store_memory_stats); INIT_COLUMNAR_SYMBOL(PGFunction, test_columnar_storage_write_new_page); + + /* + * This part is only for SECURITY LABEL tests + * mimicking what an actual security label provider would do + */ + if (RunningUnderCitusTestSuite) + { + register_label_provider("citus '!tests_label_provider", + citus_test_object_relabel); + } } @@ -2293,13 +2304,14 @@ RegisterCitusConfigVariables(void) WarnIfReplicationModelIsSet, NULL, NULL); DefineCustomBoolVariable( - "citus.running_under_isolation_test", + "citus.running_under_citus_test_suite", gettext_noop( "Only useful for testing purposes, when set to true, Citus does some " - "tricks to implement useful isolation tests with rebalancing. Should " + "tricks to implement useful isolation tests with rebalancing. It also " + "registers a dummy label provider for SECURITY LABEL tests. Should " "never be set to true on production systems "), gettext_noop("for details of the tricks implemented, refer to the source code"), - &RunningUnderIsolationTest, + &RunningUnderCitusTestSuite, false, PGC_SUSET, GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE, diff --git a/src/include/distributed/commands.h b/src/include/distributed/commands.h index 32bb3810067..caf40ff9580 100644 --- a/src/include/distributed/commands.h +++ b/src/include/distributed/commands.h @@ -510,6 +510,11 @@ extern List * AlterSchemaOwnerStmtObjectAddress(Node *node, bool missing_ok, extern List * AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok, bool isPostprocess); +/* seclabel.c - forward declarations*/ +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); + /* sequence.c - forward declarations */ extern List * PreprocessAlterSequenceStmt(Node *node, const char *queryString, ProcessUtilityContext processUtilityContext); diff --git a/src/include/distributed/deparser.h b/src/include/distributed/deparser.h index 04a96e4bee4..8c4c58c8817 100644 --- a/src/include/distributed/deparser.h +++ b/src/include/distributed/deparser.h @@ -259,6 +259,9 @@ extern void QualifyRenameTextSearchDictionaryStmt(Node *node); extern void QualifyTextSearchConfigurationCommentStmt(Node *node); extern void QualifyTextSearchDictionaryCommentStmt(Node *node); +/* forward declarations for deparse_seclabel_stmts.c */ +extern char * DeparseSecLabelStmt(Node *node); + /* forward declarations for deparse_sequence_stmts.c */ extern char * DeparseDropSequenceStmt(Node *node); extern char * DeparseRenameSequenceStmt(Node *node); diff --git a/src/include/distributed/shard_rebalancer.h b/src/include/distributed/shard_rebalancer.h index 8e47ac1e5c1..79414eb3c88 100644 --- a/src/include/distributed/shard_rebalancer.h +++ b/src/include/distributed/shard_rebalancer.h @@ -191,7 +191,7 @@ typedef struct RebalancePlanFunctions extern char *VariablesToBePassedToNewConnections; extern int MaxRebalancerLoggedIgnoredMoves; extern int RebalancerByDiskSizeBaseCost; -extern bool RunningUnderIsolationTest; +extern bool RunningUnderCitusTestSuite; extern bool PropagateSessionSettingsForLoopbackConnection; extern int MaxBackgroundTaskExecutorsPerNode; diff --git a/src/test/regress/expected/multi_test_helpers.out b/src/test/regress/expected/multi_test_helpers.out index f3a0f2bbaf8..d97c0579c77 100644 --- a/src/test/regress/expected/multi_test_helpers.out +++ b/src/test/regress/expected/multi_test_helpers.out @@ -631,3 +631,33 @@ begin end loop; end; $$; +-- Returns pg_seclabels entries from all nodes in the cluster for which +-- the object name is the input. +CREATE OR REPLACE FUNCTION get_citus_tests_label_provider_labels(object_name text, + master_port INTEGER DEFAULT 57636, + worker_1_port INTEGER DEFAULT 57637, + worker_2_port INTEGER DEFAULT 57638) +RETURNS TABLE ( + node_type text, + result text +) +AS $func$ +DECLARE + pg_seclabels_cmd TEXT := 'SELECT to_jsonb(q.*) FROM (' || + 'SELECT provider, objtype, label FROM pg_seclabels ' || + 'WHERE objname = ''' || object_name || ''') q'; +BEGIN + RETURN QUERY + SELECT + CASE + WHEN nodeport = master_port THEN 'coordinator' + WHEN nodeport = worker_1_port THEN 'worker_1' + WHEN nodeport = worker_2_port THEN 'worker_2' + ELSE 'unexpected_node' + END AS node_type, + a.result + FROM run_command_on_all_nodes(pg_seclabels_cmd) a + JOIN pg_dist_node USING (nodeid) + ORDER BY node_type; +END; +$func$ LANGUAGE plpgsql; diff --git a/src/test/regress/expected/seclabel.out b/src/test/regress/expected/seclabel.out new file mode 100644 index 00000000000..f826de44b66 --- /dev/null +++ b/src/test/regress/expected/seclabel.out @@ -0,0 +1,173 @@ +-- +-- SECLABEL +-- +-- Test suite for SECURITY LABEL ON ROLE statements +-- +-- first we remove one of the worker nodes to be able to test +-- citus_add_node later +SELECT citus_remove_node('localhost', :worker_2_port); + citus_remove_node +--------------------------------------------------------------------- + +(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 FOR "citus '!tests_label_provider" 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 FOR "citus '!tests_label_provider" ON ROLE user1 IS 'citus_unclassified'; +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", "provider": "citus '!tests_label_provider"} + worker_1 | +(2 rows) + +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 +--------------------------------------------------------------------- + +(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. +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", "provider": "citus '!tests_label_provider"} + worker_1 | +(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", "provider": "citus '!tests_label_provider"} + 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", "provider": "citus '!tests_label_provider"} + 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", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus_classified", "objtype": "table", "provider": "citus '!tests_label_provider"} +(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", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus_unclassified", "objtype": "function", "provider": "citus '!tests_label_provider"} +(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", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus_classified", "objtype": "view", "provider": "citus '!tests_label_provider"} +(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%'; +-- we have exactly one provider loaded, so we may not include the provider in the command +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 +SECURITY LABEL ON ROLE user1 IS NULL; +NOTICE: issuing SECURITY LABEL ON ROLE user1 IS NULL +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx +SECURITY LABEL ON ROLE user1 IS 'citus_unclassified'; +NOTICE: issuing SECURITY LABEL 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 ''!unclassified'; +NOTICE: issuing SECURITY LABEL FOR "citus '!tests_label_provider" ON ROLE "user 2" IS 'citus ''!unclassified' +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') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus_unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus_unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} +(2 rows) + +SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type; + node_type | result +--------------------------------------------------------------------- + coordinator | {"label": "citus '!unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus '!unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} +(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 ''!unclassified' +DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx + ?column? +--------------------------------------------------------------------- + 1 +(1 row) + +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", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus_unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} + worker_2 | {"label": "citus_unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} +(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 '!unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} + worker_1 | {"label": "citus '!unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} + worker_2 | {"label": "citus '!unclassified", "objtype": "role", "provider": "citus '!tests_label_provider"} +(3 rows) + +-- cleanup +RESET citus.log_remote_commands; +DROP ROLE user1, "user 2"; diff --git a/src/test/regress/multi_1_schedule b/src/test/regress/multi_1_schedule index 4dead5be390..c996b5c02a7 100644 --- a/src/test/regress/multi_1_schedule +++ b/src/test/regress/multi_1_schedule @@ -31,6 +31,7 @@ test: propagate_extension_commands test: escape_extension_name test: ref_citus_local_fkeys test: alter_database_owner +test: seclabel test: distributed_triggers test: create_single_shard_table # don't parallelize single_shard_table_udfs to make sure colocation ids are sequential diff --git a/src/test/regress/pg_regress_multi.pl b/src/test/regress/pg_regress_multi.pl index adcb431e13d..66016f7c62b 100755 --- a/src/test/regress/pg_regress_multi.pl +++ b/src/test/regress/pg_regress_multi.pl @@ -512,6 +512,12 @@ sub generate_hba # we disable some restrictions for local objects like local views to not break postgres vanilla test behaviour. push(@pgOptions, "citus.enforce_object_restrictions_for_local_objects=false"); } +else +{ + # We currently need this config for isolation tests and security label tests + # this option loads a security label provider, which we don't want in vanilla tests + push(@pgOptions, "citus.running_under_citus_test_suite=true"); +} if ($useMitmproxy) { @@ -562,7 +568,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_isolation_test=true"); # Disable all features of the maintenance daemon. Otherwise queries might # randomly show temporarily as "waiting..." because they are waiting for the diff --git a/src/test/regress/sql/multi_test_helpers.sql b/src/test/regress/sql/multi_test_helpers.sql index 57abf15a35c..17fcf58bb7d 100644 --- a/src/test/regress/sql/multi_test_helpers.sql +++ b/src/test/regress/sql/multi_test_helpers.sql @@ -660,3 +660,33 @@ begin end loop; end; $$; +-- Returns pg_seclabels entries from all nodes in the cluster for which +-- the object name is the input. +CREATE OR REPLACE FUNCTION get_citus_tests_label_provider_labels(object_name text, + master_port INTEGER DEFAULT 57636, + worker_1_port INTEGER DEFAULT 57637, + worker_2_port INTEGER DEFAULT 57638) +RETURNS TABLE ( + node_type text, + result text +) +AS $func$ +DECLARE + pg_seclabels_cmd TEXT := 'SELECT to_jsonb(q.*) FROM (' || + 'SELECT provider, objtype, label FROM pg_seclabels ' || + 'WHERE objname = ''' || object_name || ''') q'; +BEGIN + RETURN QUERY + SELECT + CASE + WHEN nodeport = master_port THEN 'coordinator' + WHEN nodeport = worker_1_port THEN 'worker_1' + WHEN nodeport = worker_2_port THEN 'worker_2' + ELSE 'unexpected_node' + END AS node_type, + a.result + FROM run_command_on_all_nodes(pg_seclabels_cmd) a + JOIN pg_dist_node USING (nodeid) + ORDER BY node_type; +END; +$func$ LANGUAGE plpgsql; diff --git a/src/test/regress/sql/seclabel.sql b/src/test/regress/sql/seclabel.sql new file mode 100644 index 00000000000..e523fc1dacd --- /dev/null +++ b/src/test/regress/sql/seclabel.sql @@ -0,0 +1,87 @@ +-- +-- SECLABEL +-- +-- Test suite for SECURITY LABEL ON ROLE statements +-- + +-- first we remove one of the worker nodes to be able to test +-- 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 FOR "citus '!tests_label_provider" 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 FOR "citus '!tests_label_provider" ON ROLE user1 IS 'citus_unclassified'; +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'; +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%'; + +-- we have exactly one provider loaded, so we may not include the provider in the command +SECURITY LABEL for "citus '!tests_label_provider" ON ROLE user1 IS 'citus_classified'; +SECURITY LABEL ON ROLE user1 IS NULL; +SECURITY LABEL ON ROLE user1 IS 'citus_unclassified'; +SECURITY LABEL for "citus '!tests_label_provider" ON ROLE "user 2" IS 'citus ''!unclassified'; + +\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') ORDER BY node_type; +SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type; + +-- 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') 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, "user 2";