mirror of https://github.com/citusdata/citus.git
Propagates SECURITY LABEL ON ROLE stmt (#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.pull/7350/head
parent
c6fbb72c02
commit
0d1f18862b
|
@ -5,6 +5,6 @@ set -euo pipefail
|
||||||
source ci/ci_helpers.sh
|
source ci/ci_helpers.sh
|
||||||
|
|
||||||
# extract citus gucs in the form of "citus.X"
|
# 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
|
sort -c gucs.out
|
||||||
rm gucs.out
|
rm 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",
|
|
@ -374,6 +374,15 @@ static DistributeObjectOps Any_Rename = {
|
||||||
.address = NULL,
|
.address = NULL,
|
||||||
.markDistributed = false,
|
.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 = {
|
static DistributeObjectOps Attribute_Rename = {
|
||||||
.deparse = DeparseRenameAttributeStmt,
|
.deparse = DeparseRenameAttributeStmt,
|
||||||
.qualify = QualifyRenameAttributeStmt,
|
.qualify = QualifyRenameAttributeStmt,
|
||||||
|
@ -2020,6 +2029,11 @@ GetDistributeObjectOps(Node *node)
|
||||||
return &Vacuum_Analyze;
|
return &Vacuum_Analyze;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
case T_SecLabelStmt:
|
||||||
|
{
|
||||||
|
return &Any_SecLabel;
|
||||||
|
}
|
||||||
|
|
||||||
case T_RenameStmt:
|
case T_RenameStmt:
|
||||||
{
|
{
|
||||||
RenameStmt *stmt = castNode(RenameStmt, node);
|
RenameStmt *stmt = castNode(RenameStmt, node);
|
||||||
|
|
|
@ -23,6 +23,7 @@
|
||||||
#include "catalog/pg_auth_members.h"
|
#include "catalog/pg_auth_members.h"
|
||||||
#include "catalog/pg_authid.h"
|
#include "catalog/pg_authid.h"
|
||||||
#include "catalog/pg_db_role_setting.h"
|
#include "catalog/pg_db_role_setting.h"
|
||||||
|
#include "catalog/pg_shseclabel.h"
|
||||||
#include "catalog/pg_type.h"
|
#include "catalog/pg_type.h"
|
||||||
#include "catalog/objectaddress.h"
|
#include "catalog/objectaddress.h"
|
||||||
#include "commands/dbcommands.h"
|
#include "commands/dbcommands.h"
|
||||||
|
@ -65,6 +66,7 @@ static DefElem * makeDefElemBool(char *name, bool value);
|
||||||
static List * GenerateRoleOptionsList(HeapTuple tuple);
|
static List * GenerateRoleOptionsList(HeapTuple tuple);
|
||||||
static List * GenerateGrantRoleStmtsFromOptions(RoleSpec *roleSpec, List *options);
|
static List * GenerateGrantRoleStmtsFromOptions(RoleSpec *roleSpec, List *options);
|
||||||
static List * GenerateGrantRoleStmtsOfRole(Oid roleid);
|
static List * GenerateGrantRoleStmtsOfRole(Oid roleid);
|
||||||
|
static List * GenerateSecLabelOnRoleStmts(Oid roleid, char *rolename);
|
||||||
static void EnsureSequentialModeForRoleDDL(void);
|
static void EnsureSequentialModeForRoleDDL(void);
|
||||||
|
|
||||||
static char * GetRoleNameFromDbRoleSetting(HeapTuple tuple,
|
static char * GetRoleNameFromDbRoleSetting(HeapTuple tuple,
|
||||||
|
@ -515,13 +517,14 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid)
|
||||||
{
|
{
|
||||||
HeapTuple roleTuple = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleOid));
|
HeapTuple roleTuple = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleOid));
|
||||||
Form_pg_authid role = ((Form_pg_authid) GETSTRUCT(roleTuple));
|
Form_pg_authid role = ((Form_pg_authid) GETSTRUCT(roleTuple));
|
||||||
|
char *rolename = pstrdup(NameStr(role->rolname));
|
||||||
|
|
||||||
CreateRoleStmt *createRoleStmt = NULL;
|
CreateRoleStmt *createRoleStmt = NULL;
|
||||||
if (EnableCreateRolePropagation)
|
if (EnableCreateRolePropagation)
|
||||||
{
|
{
|
||||||
createRoleStmt = makeNode(CreateRoleStmt);
|
createRoleStmt = makeNode(CreateRoleStmt);
|
||||||
createRoleStmt->stmt_type = ROLESTMT_ROLE;
|
createRoleStmt->stmt_type = ROLESTMT_ROLE;
|
||||||
createRoleStmt->role = pstrdup(NameStr(role->rolname));
|
createRoleStmt->role = rolename;
|
||||||
createRoleStmt->options = GenerateRoleOptionsList(roleTuple);
|
createRoleStmt->options = GenerateRoleOptionsList(roleTuple);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -532,7 +535,7 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid)
|
||||||
alterRoleStmt->role = makeNode(RoleSpec);
|
alterRoleStmt->role = makeNode(RoleSpec);
|
||||||
alterRoleStmt->role->roletype = ROLESPEC_CSTRING;
|
alterRoleStmt->role->roletype = ROLESPEC_CSTRING;
|
||||||
alterRoleStmt->role->location = -1;
|
alterRoleStmt->role->location = -1;
|
||||||
alterRoleStmt->role->rolename = pstrdup(NameStr(role->rolname));
|
alterRoleStmt->role->rolename = rolename;
|
||||||
alterRoleStmt->action = 1;
|
alterRoleStmt->action = 1;
|
||||||
alterRoleStmt->options = GenerateRoleOptionsList(roleTuple);
|
alterRoleStmt->options = GenerateRoleOptionsList(roleTuple);
|
||||||
}
|
}
|
||||||
|
@ -544,7 +547,7 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid)
|
||||||
{
|
{
|
||||||
/* add a worker_create_or_alter_role command if any of them are set */
|
/* add a worker_create_or_alter_role command if any of them are set */
|
||||||
char *createOrAlterRoleQuery = CreateCreateOrAlterRoleCommand(
|
char *createOrAlterRoleQuery = CreateCreateOrAlterRoleCommand(
|
||||||
pstrdup(NameStr(role->rolname)),
|
rolename,
|
||||||
createRoleStmt,
|
createRoleStmt,
|
||||||
alterRoleStmt);
|
alterRoleStmt);
|
||||||
|
|
||||||
|
@ -566,6 +569,20 @@ GenerateCreateOrAlterRoleCommand(Oid roleOid)
|
||||||
{
|
{
|
||||||
completeRoleList = lappend(completeRoleList, DeparseTreeNode(stmt));
|
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_ptr(stmt, secLabelOnRoleStmts)
|
||||||
|
{
|
||||||
|
completeRoleList = lappend(completeRoleList, DeparseTreeNode(stmt));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
return completeRoleList;
|
return completeRoleList;
|
||||||
|
@ -895,6 +912,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
|
* 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
|
* role that is being created. With that query we can create the role in the
|
||||||
|
|
|
@ -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_sync.h"
|
||||||
|
#include "distributed/metadata/distobject.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)));
|
||||||
|
}
|
|
@ -0,0 +1,78 @@
|
||||||
|
/*-------------------------------------------------------------------------
|
||||||
|
*
|
||||||
|
* deparse_seclabel_stmts.c
|
||||||
|
* All routines to deparse SECURITY LABEL statements.
|
||||||
|
*
|
||||||
|
* Copyright (c), Citus Data, Inc.
|
||||||
|
*
|
||||||
|
*-------------------------------------------------------------------------
|
||||||
|
*/
|
||||||
|
|
||||||
|
#include "postgres.h"
|
||||||
|
|
||||||
|
#include "distributed/deparser.h"
|
||||||
|
#include "nodes/parsenodes.h"
|
||||||
|
#include "utils/builtins.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");
|
||||||
|
}
|
||||||
|
}
|
|
@ -317,7 +317,7 @@ PG_FUNCTION_INFO_V1(citus_rebalance_start);
|
||||||
PG_FUNCTION_INFO_V1(citus_rebalance_stop);
|
PG_FUNCTION_INFO_V1(citus_rebalance_stop);
|
||||||
PG_FUNCTION_INFO_V1(citus_rebalance_wait);
|
PG_FUNCTION_INFO_V1(citus_rebalance_wait);
|
||||||
|
|
||||||
bool RunningUnderIsolationTest = false;
|
bool RunningUnderCitusTestSuite = false;
|
||||||
int MaxRebalancerLoggedIgnoredMoves = 5;
|
int MaxRebalancerLoggedIgnoredMoves = 5;
|
||||||
int RebalancerByDiskSizeBaseCost = 100 * 1024 * 1024;
|
int RebalancerByDiskSizeBaseCost = 100 * 1024 * 1024;
|
||||||
bool PropagateSessionSettingsForLoopbackConnection = false;
|
bool PropagateSessionSettingsForLoopbackConnection = false;
|
||||||
|
|
|
@ -1143,7 +1143,7 @@ ConflictWithIsolationTestingBeforeCopy(void)
|
||||||
const bool sessionLock = false;
|
const bool sessionLock = false;
|
||||||
const bool dontWait = false;
|
const bool dontWait = false;
|
||||||
|
|
||||||
if (RunningUnderIsolationTest)
|
if (RunningUnderCitusTestSuite)
|
||||||
{
|
{
|
||||||
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId,
|
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId,
|
||||||
SHARD_MOVE_ADVISORY_LOCK_SECOND_KEY,
|
SHARD_MOVE_ADVISORY_LOCK_SECOND_KEY,
|
||||||
|
@ -1177,7 +1177,7 @@ ConflictWithIsolationTestingAfterCopy(void)
|
||||||
const bool sessionLock = false;
|
const bool sessionLock = false;
|
||||||
const bool dontWait = false;
|
const bool dontWait = false;
|
||||||
|
|
||||||
if (RunningUnderIsolationTest)
|
if (RunningUnderCitusTestSuite)
|
||||||
{
|
{
|
||||||
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId,
|
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId,
|
||||||
SHARD_MOVE_ADVISORY_LOCK_FIRST_KEY,
|
SHARD_MOVE_ADVISORY_LOCK_FIRST_KEY,
|
||||||
|
|
|
@ -29,6 +29,7 @@
|
||||||
#include "citus_version.h"
|
#include "citus_version.h"
|
||||||
#include "commands/explain.h"
|
#include "commands/explain.h"
|
||||||
#include "commands/extension.h"
|
#include "commands/extension.h"
|
||||||
|
#include "commands/seclabel.h"
|
||||||
#include "common/string.h"
|
#include "common/string.h"
|
||||||
#include "executor/executor.h"
|
#include "executor/executor.h"
|
||||||
#include "distributed/backend_data.h"
|
#include "distributed/backend_data.h"
|
||||||
|
@ -574,6 +575,16 @@ _PG_init(void)
|
||||||
INIT_COLUMNAR_SYMBOL(PGFunction, columnar_storage_info);
|
INIT_COLUMNAR_SYMBOL(PGFunction, columnar_storage_info);
|
||||||
INIT_COLUMNAR_SYMBOL(PGFunction, columnar_store_memory_stats);
|
INIT_COLUMNAR_SYMBOL(PGFunction, columnar_store_memory_stats);
|
||||||
INIT_COLUMNAR_SYMBOL(PGFunction, test_columnar_storage_write_new_page);
|
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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -2305,13 +2316,14 @@ RegisterCitusConfigVariables(void)
|
||||||
WarnIfReplicationModelIsSet, NULL, NULL);
|
WarnIfReplicationModelIsSet, NULL, NULL);
|
||||||
|
|
||||||
DefineCustomBoolVariable(
|
DefineCustomBoolVariable(
|
||||||
"citus.running_under_isolation_test",
|
"citus.running_under_citus_test_suite",
|
||||||
gettext_noop(
|
gettext_noop(
|
||||||
"Only useful for testing purposes, when set to true, Citus does some "
|
"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 "),
|
"never be set to true on production systems "),
|
||||||
gettext_noop("for details of the tricks implemented, refer to the source code"),
|
gettext_noop("for details of the tricks implemented, refer to the source code"),
|
||||||
&RunningUnderIsolationTest,
|
&RunningUnderCitusTestSuite,
|
||||||
false,
|
false,
|
||||||
PGC_SUSET,
|
PGC_SUSET,
|
||||||
GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE,
|
GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL | GUC_NOT_IN_SAMPLE,
|
||||||
|
|
|
@ -521,6 +521,11 @@ extern List * AlterSchemaOwnerStmtObjectAddress(Node *node, bool missing_ok,
|
||||||
extern List * AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok, bool
|
extern List * AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok, bool
|
||||||
isPostprocess);
|
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 */
|
/* sequence.c - forward declarations */
|
||||||
extern List * PreprocessAlterSequenceStmt(Node *node, const char *queryString,
|
extern List * PreprocessAlterSequenceStmt(Node *node, const char *queryString,
|
||||||
ProcessUtilityContext processUtilityContext);
|
ProcessUtilityContext processUtilityContext);
|
||||||
|
|
|
@ -260,6 +260,9 @@ extern void QualifyRenameTextSearchDictionaryStmt(Node *node);
|
||||||
extern void QualifyTextSearchConfigurationCommentStmt(Node *node);
|
extern void QualifyTextSearchConfigurationCommentStmt(Node *node);
|
||||||
extern void QualifyTextSearchDictionaryCommentStmt(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 */
|
/* forward declarations for deparse_sequence_stmts.c */
|
||||||
extern char * DeparseDropSequenceStmt(Node *node);
|
extern char * DeparseDropSequenceStmt(Node *node);
|
||||||
extern char * DeparseRenameSequenceStmt(Node *node);
|
extern char * DeparseRenameSequenceStmt(Node *node);
|
||||||
|
|
|
@ -189,7 +189,7 @@ typedef struct RebalancePlanFunctions
|
||||||
extern char *VariablesToBePassedToNewConnections;
|
extern char *VariablesToBePassedToNewConnections;
|
||||||
extern int MaxRebalancerLoggedIgnoredMoves;
|
extern int MaxRebalancerLoggedIgnoredMoves;
|
||||||
extern int RebalancerByDiskSizeBaseCost;
|
extern int RebalancerByDiskSizeBaseCost;
|
||||||
extern bool RunningUnderIsolationTest;
|
extern bool RunningUnderCitusTestSuite;
|
||||||
extern bool PropagateSessionSettingsForLoopbackConnection;
|
extern bool PropagateSessionSettingsForLoopbackConnection;
|
||||||
extern int MaxBackgroundTaskExecutorsPerNode;
|
extern int MaxBackgroundTaskExecutorsPerNode;
|
||||||
|
|
||||||
|
|
|
@ -526,3 +526,33 @@ BEGIN
|
||||||
RETURN result;
|
RETURN result;
|
||||||
END;
|
END;
|
||||||
$func$ LANGUAGE plpgsql;
|
$func$ LANGUAGE plpgsql;
|
||||||
|
-- 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;
|
||||||
|
|
|
@ -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";
|
|
@ -32,6 +32,7 @@ test: propagate_extension_commands
|
||||||
test: escape_extension_name
|
test: escape_extension_name
|
||||||
test: ref_citus_local_fkeys
|
test: ref_citus_local_fkeys
|
||||||
test: alter_database_owner
|
test: alter_database_owner
|
||||||
|
test: seclabel
|
||||||
test: distributed_triggers
|
test: distributed_triggers
|
||||||
test: create_single_shard_table
|
test: create_single_shard_table
|
||||||
# don't parallelize single_shard_table_udfs to make sure colocation ids are sequential
|
# don't parallelize single_shard_table_udfs to make sure colocation ids are sequential
|
||||||
|
|
|
@ -510,6 +510,12 @@ if($vanillatest)
|
||||||
# we disable some restrictions for local objects like local views to not break postgres vanilla test behaviour.
|
# 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");
|
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)
|
if ($useMitmproxy)
|
||||||
{
|
{
|
||||||
|
@ -560,7 +566,6 @@ if($isolationtester)
|
||||||
push(@pgOptions, "citus.metadata_sync_interval=1000");
|
push(@pgOptions, "citus.metadata_sync_interval=1000");
|
||||||
push(@pgOptions, "citus.metadata_sync_retry_interval=100");
|
push(@pgOptions, "citus.metadata_sync_retry_interval=100");
|
||||||
push(@pgOptions, "client_min_messages='warning'"); # pg12 introduced notice showing during isolation tests
|
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
|
# Disable all features of the maintenance daemon. Otherwise queries might
|
||||||
# randomly show temporarily as "waiting..." because they are waiting for the
|
# randomly show temporarily as "waiting..." because they are waiting for the
|
||||||
|
|
|
@ -550,3 +550,34 @@ BEGIN
|
||||||
RETURN result;
|
RETURN result;
|
||||||
END;
|
END;
|
||||||
$func$ LANGUAGE plpgsql;
|
$func$ LANGUAGE plpgsql;
|
||||||
|
|
||||||
|
-- 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;
|
||||||
|
|
|
@ -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";
|
Loading…
Reference in New Issue