Address reviews

pull/7304/head
naisila 2023-11-14 11:17:43 +03:00
parent 4f028008b4
commit 4825d24241
No known key found for this signature in database
GPG Key ID: A824BA9862D73E6D
10 changed files with 179 additions and 238 deletions

View File

@ -1,134 +0,0 @@
"citus_tests_label_provider",
"citus.all_modifications_commutative",
"citus.allow_modifications_from_workers_to_replicated_tables",
"citus.allow_nested_distributed_execution",
"citus.allow_unsafe_constraints",
"citus.allow_unsafe_locks_from_workers",
"citus.background_task_queue_interval",
"citus.check_available_space_before_move",
"citus.cluster_name",
"citus.coordinator_aggregation_strategy",
"citus.copy_switchover_threshold",
"citus.count_distinct_error_rate",
"citus.cpu_priority",
"citus.cpu_priority_for_logical_replication_senders",
"citus.create_object_propagation",
"citus.defer_drop_after_shard_move",
"citus.defer_drop_after_shard_split",
"citus.defer_shard_delete_interval",
"citus.desired_percent_disk_available_after_move",
"citus.distributed_deadlock_detection_factor",
"citus.enable_alter_database_owner",
"citus.enable_alter_role_propagation",
"citus.enable_alter_role_set_propagation",
"citus.enable_binary_protocol",
"citus.enable_change_data_capture",
"citus.enable_cluster_clock",
"citus.enable_cost_based_connection_establishment",
"citus.enable_create_role_propagation",
"citus.enable_create_type_propagation",
"citus.enable_ddl_propagation",
"citus.enable_deadlock_prevention",
"citus.enable_fast_path_router_planner",
"citus.enable_local_execution",
"citus.enable_local_reference_table_foreign_keys",
"citus.enable_manual_changes_to_shards",
"citus.enable_manual_metadata_changes_for_user",
"citus.enable_metadata_sync",
"citus.enable_non_colocated_router_query_pushdown",
"citus.enable_repartition_joins",
"citus.enable_repartitioned_insert_select",
"citus.enable_router_execution",
"citus.enable_schema_based_sharding",
"citus.enable_single_hash_repartition_joins",
"citus.enable_statistics_collection",
"citus.enable_unique_job_ids",
"citus.enable_unsafe_triggers",
"citus.enable_unsupported_feature_messages",
"citus.enable_version_checks",
"citus.enforce_foreign_key_restrictions",
"citus.enforce_object_restrictions_for_local_objects",
"citus.executor_slow_start_interval",
"citus.explain_all_tasks",
"citus.explain_analyze_sort_method",
"citus.explain_distributed_queries",
"citus.force_max_query_parallelization",
"citus.function_opens_transaction_block",
"citus.grep_remote_commands",
"citus.hide_citus_dependent_objects",
"citus.hide_shards_from_app_name_prefixes",
"citus.isolation_test_session_process_id",
"citus.isolation_test_session_remote_process_id",
"citus.limit_clause_row_fetch_count",
"citus.local_copy_flush_threshold",
"citus.local_hostname",
"citus.local_shared_pool_size",
"citus.local_table_join_policy",
"citus.log_distributed_deadlock_detection",
"citus.log_intermediate_results",
"citus.log_local_commands",
"citus.log_multi_join_order",
"citus.log_remote_commands",
"citus.logical_replication_timeout",
"citus.main_db",
"citus.max_adaptive_executor_pool_size",
"citus.max_background_task_executors",
"citus.max_background_task_executors_per_node",
"citus.max_cached_connection_lifetime",
"citus.max_cached_conns_per_worker",
"citus.max_client_connections",
"citus.max_high_priority_background_processes",
"citus.max_intermediate_result_size",
"citus.max_matview_size_to_auto_recreate",
"citus.max_rebalancer_logged_ignored_moves",
"citus.max_shared_pool_size",
"citus.max_worker_nodes_tracked",
"citus.metadata_sync_interval",
"citus.metadata_sync_mode",
"citus.metadata_sync_retry_interval",
"citus.mitmfifo",
"citus.multi_shard_modify_mode",
"citus.multi_task_query_log_level",
"citus.next_cleanup_record_id",
"citus.next_operation_id",
"citus.next_placement_id",
"citus.next_shard_id",
"citus.node_connection_timeout",
"citus.node_conninfo",
"citus.override_table_visibility",
"citus.prevent_incomplete_connection_establishment",
"citus.propagate_session_settings_for_loopback_connection",
"citus.propagate_set_commands",
"citus.rebalancer_by_disk_size_base_cost",
"citus.recover_2pc_interval",
"citus.remote_copy_flush_threshold",
"citus.remote_task_check_interval",
"citus.repartition_join_bucket_count_per_node",
"citus.replicate_reference_tables_on_activate",
"citus.replication_model",
"citus.running_under_citus_test_suite",
"citus.select_opens_transaction_block",
"citus.shard_count",
"citus.shard_replication_factor",
"citus.show_shards_for_app_name_prefixes",
"citus.skip_advisory_lock_permission_checks",
"citus.skip_constraint_validation",
"citus.skip_jsonb_validation_in_copy",
"citus.sort_returning",
"citus.stat_statements_max",
"citus.stat_statements_purge_interval",
"citus.stat_statements_track",
"citus.stat_tenants_limit",
"citus.stat_tenants_log_level",
"citus.stat_tenants_period",
"citus.stat_tenants_track",
"citus.stat_tenants_untracked_sample_rate",
"citus.subquery_pushdown",
"citus.task_assignment_policy",
"citus.task_executor_type",
"citus.use_citus_managed_tables",
"citus.use_secondary_nodes",
"citus.values_materialization_threshold",
"citus.version",
"citus.worker_min_messages",
"citus.writable_standby_coordinator",

View File

@ -377,7 +377,7 @@ static DistributeObjectOps Any_Rename = {
static DistributeObjectOps Any_SecLabel = {
.deparse = DeparseSecLabelStmt,
.qualify = NULL,
.preprocess = PreprocessSecLabelStmt,
.preprocess = NULL,
.postprocess = PostprocessSecLabelStmt,
.operationType = DIST_OPS_ALTER,
.address = SecLabelStmtObjectAddress,

View File

@ -937,7 +937,7 @@ GenerateSecLabelOnRoleStmts(Oid roleid, char *rolename)
{
SecLabelStmt *secLabelStmt = makeNode(SecLabelStmt);
secLabelStmt->objtype = OBJECT_ROLE;
secLabelStmt->object = (Node *) makeString(rolename);
secLabelStmt->object = (Node *) makeString(pstrdup(rolename));
Datum datumArray[Natts_pg_shseclabel];
bool isNullArray[Natts_pg_shseclabel];

View File

@ -19,18 +19,17 @@
#include "distributed/metadata_sync.h"
#include "distributed/metadata/distobject.h"
/*
* PreprocessSecLabelStmt is executed before the statement is applied to the local
* postgres instance.
*
* In this stage we can prepare the commands that need to be run on all workers to assign
* PostprocessSecLabelStmt prepares the commands that need to be run on all workers to assign
* security labels on distributed objects, currently supporting just Role objects.
* It also ensures that all object dependencies exist on all
* nodes for the object in the SecLabelStmt.
*/
List *
PreprocessSecLabelStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext)
PostprocessSecLabelStmt(Node *node, const char *queryString)
{
if (!IsCoordinator() || !ShouldPropagate())
if (!ShouldPropagate())
{
return NIL;
}
@ -45,13 +44,17 @@ PreprocessSecLabelStmt(Node *node, const char *queryString,
if (secLabelStmt->objtype != OBJECT_ROLE)
{
if (EnableUnsupportedFeatureMessages)
/*
* If we are not in the coordinator, we don't want to interrupt the security
* label command with notices, the user expects that from the worker node
* the command will not be propagated
*/
if (EnableUnsupportedFeatureMessages && IsCoordinator())
{
ereport(NOTICE, (errmsg("not propagating SECURITY LABEL commands whose "
"object type is not role"),
errhint("Connect to worker nodes directly to manually "
"run the same SECURITY LABEL command after "
"disabling DDL propagation.")));
"run the same SECURITY LABEL command.")));
}
return NIL;
}
@ -61,6 +64,9 @@ PreprocessSecLabelStmt(Node *node, const char *queryString,
return NIL;
}
EnsureCoordinator();
EnsureAllObjectDependenciesExistOnAllNodes(objectAddresses);
const char *sql = DeparseTreeNode((Node *) secLabelStmt);
List *commandList = list_make3(DISABLE_DDL_PROPAGATION,
@ -71,36 +77,6 @@ PreprocessSecLabelStmt(Node *node, const char *queryString,
}
/*
* PostprocessSecLabelStmt ensures that all object dependencies exist on all
* nodes for the object in the SecLabelStmt. Currently, we only support SecLabelStmts
* operating on a ROLE object.
*/
List *
PostprocessSecLabelStmt(Node *node, const char *queryString)
{
if (!EnableCreateRolePropagation || !IsCoordinator() || !ShouldPropagate())
{
return NIL;
}
SecLabelStmt *secLabelStmt = castNode(SecLabelStmt, node);
if (secLabelStmt->objtype != OBJECT_ROLE)
{
return NIL;
}
List *objectAddresses = GetObjectAddressListFromParseTree(node, false, false);
if (IsAnyObjectDistributed(objectAddresses))
{
EnsureAllObjectDependenciesExistOnAllNodes(objectAddresses);
}
return NIL;
}
/*
* SecLabelStmtObjectAddress returns the object address of the object on
* which this statement operates (secLabelStmt->object). Note that it has no limitation
@ -136,8 +112,8 @@ 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 ||
strcmp(seclabel, "citus classified") == 0)
{
return;
}

View File

@ -44,7 +44,7 @@ AppendSecLabelStmt(StringInfo buf, SecLabelStmt *stmt)
if (stmt->provider != NULL)
{
appendStringInfo(buf, "FOR %s ", stmt->provider);
appendStringInfo(buf, "FOR %s ", quote_identifier(stmt->provider));
}
appendStringInfoString(buf, "ON ");
@ -53,7 +53,7 @@ AppendSecLabelStmt(StringInfo buf, SecLabelStmt *stmt)
{
case OBJECT_ROLE:
{
appendStringInfo(buf, "ROLE %s ", strVal(stmt->object));
appendStringInfo(buf, "ROLE %s ", quote_identifier(strVal(stmt->object)));
break;
}

View File

@ -522,8 +522,6 @@ extern List * AlterSchemaRenameStmtObjectAddress(Node *node, bool missing_ok, bo
isPostprocess);
/* seclabel.c - forward declarations*/
extern List * PreprocessSecLabelStmt(Node *node, const char *queryString,
ProcessUtilityContext processUtilityContext);
extern List * PostprocessSecLabelStmt(Node *node, const char *queryString);
extern List * SecLabelStmtObjectAddress(Node *node, bool missing_ok, bool isPostprocess);
extern void citus_test_object_relabel(const ObjectAddress *object, const char *seclabel);

View File

@ -101,7 +101,7 @@ check-base: all
# check-minimal only sets up the cluster
check-minimal: all
$(pg_regress_multi_check) --load-extension=citus --seclabel \
$(pg_regress_multi_check) --load-extension=citus \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/minimal_schedule $(EXTRA_TESTS)
check-base-vg: all
@ -118,7 +118,7 @@ check-minimal-mx: all
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/mx_minimal_schedule $(EXTRA_TESTS)
check-custom-schedule: all
$(pg_regress_multi_check) --load-extension=citus --seclabel-test --worker-count=$(WORKERCOUNT) \
$(pg_regress_multi_check) --load-extension=citus --worker-count=$(WORKERCOUNT) \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/$(SCHEDULE) $(EXTRA_TESTS)
check-failure-custom-schedule: all
@ -159,7 +159,7 @@ check-enterprise: all
$(pg_regress_multi_check) --load-extension=citus \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/enterprise_schedule $(EXTRA_TESTS)
check-multi-1: all
$(pg_regress_multi_check) --load-extension=citus --seclabel-test \
$(pg_regress_multi_check) --load-extension=citus \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_1_schedule $(EXTRA_TESTS)
check-multi-hyperscale: all

View File

@ -11,23 +11,26 @@ SELECT citus_remove_node('localhost', :worker_2_port);
(1 row)
-- create two roles, one with characters that need escaping
CREATE ROLE user1;
CREATE ROLE "user 2";
-- check an invalid label for our current dummy hook citus_test_object_relabel
SECURITY LABEL ON ROLE user1 IS 'invalid_label';
ERROR: 'invalid_label' is not a valid security label for Citus tests.
-- if we disable metadata_sync, the command will not be propagated
SET citus.enable_metadata_sync TO off;
SECURITY LABEL ON ROLE user1 IS 'citus_unclassified';
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1');
SECURITY LABEL 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"}
coordinator | {"label": "citus unclassified", "objtype": "role"}
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
@ -35,59 +38,135 @@ SELECT create_distributed_table('a', 'a');
(1 row)
SECURITY LABEL ON TABLE a IS 'citus_classified';
-- distributed view
CREATE VIEW v_dist AS SELECT * FROM a;
-- distributed function
CREATE FUNCTION notice(text) RETURNS void LANGUAGE plpgsql AS $$
BEGIN RAISE NOTICE '%', $1; END; $$;
SECURITY LABEL ON TABLE a IS 'citus classified';
NOTICE: not propagating SECURITY LABEL commands whose object type is not role
HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command after disabling DDL propagation.
SELECT node_type, result FROM get_citus_tests_label_provider_labels('a');
HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command.
SECURITY LABEL ON FUNCTION notice IS 'citus unclassified';
NOTICE: not propagating SECURITY LABEL commands whose object type is not role
HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command.
SECURITY LABEL ON VIEW v_dist IS 'citus classified';
NOTICE: not propagating SECURITY LABEL commands whose object type is not role
HINT: Connect to worker nodes directly to manually run the same SECURITY LABEL command.
SELECT node_type, result FROM get_citus_tests_label_provider_labels('a') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus_classified", "objtype": "table"}
coordinator | {"label": "citus classified", "objtype": "table"}
worker_1 |
(2 rows)
DROP TABLE a;
SELECT node_type, result FROM get_citus_tests_label_provider_labels('notice(text)') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus unclassified", "objtype": "function"}
worker_1 |
(2 rows)
SELECT node_type, result FROM get_citus_tests_label_provider_labels('v_dist') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus classified", "objtype": "view"}
worker_1 |
(2 rows)
\c - - - :worker_1_port
SECURITY LABEL ON TABLE a IS 'citus classified';
SECURITY LABEL ON FUNCTION notice IS 'citus unclassified';
SECURITY LABEL ON VIEW v_dist IS 'citus classified';
\c - - - :master_port
SELECT node_type, result FROM get_citus_tests_label_provider_labels('a') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus classified", "objtype": "table"}
worker_1 | {"label": "citus classified", "objtype": "table"}
(2 rows)
SELECT node_type, result FROM get_citus_tests_label_provider_labels('notice(text)') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus unclassified", "objtype": "function"}
worker_1 | {"label": "citus unclassified", "objtype": "function"}
(2 rows)
SELECT node_type, result FROM get_citus_tests_label_provider_labels('v_dist') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus classified", "objtype": "view"}
worker_1 | {"label": "citus classified", "objtype": "view"}
(2 rows)
DROP TABLE a CASCADE;
NOTICE: drop cascades to view v_dist
DROP FUNCTION notice;
-- test that SECURITY LABEL statement is actually propagated for ROLES
SET citus.log_remote_commands TO on;
SET citus.grep_remote_commands = '%SECURITY LABEL%';
-- then we run a security label statement which will use the same connection to the worker node
-- it should finish successfully
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_classified';
NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus_classified'
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 for citus_tests_label_provider ON ROLE user1 IS NULL;
NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS NULL
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified';
NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified'
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus unclassified';
NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus unclassified'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
SECURITY LABEL for citus_tests_label_provider ON ROLE "user 2" IS 'citus classified';
NOTICE: issuing SECURITY LABEL FOR citus_tests_label_provider ON ROLE "user 2" IS 'citus classified'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
\c - - - :worker_1_port
-- command not allowed from worker node
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus unclassified';
ERROR: operation is not allowed on this node
HINT: Connect to the coordinator and run it again.
\c - - - :master_port
RESET citus.log_remote_commands;
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1');
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus_unclassified", "objtype": "role"}
worker_1 | {"label": "citus_unclassified", "objtype": "role"}
coordinator | {"label": "citus unclassified", "objtype": "role"}
worker_1 | {"label": "citus unclassified", "objtype": "role"}
(2 rows)
-- adding a new node will fail because the label provider is not there
-- however, this is enough for testing as we can see that the SECURITY LABEL commands
-- will be propagated when adding a new node
SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus classified", "objtype": "role"}
worker_1 | {"label": "citus classified", "objtype": "role"}
(2 rows)
-- add a new node and check that it also propagates the SECURITY LABEL statement to the new node
SET citus.log_remote_commands TO on;
SET citus.grep_remote_commands = '%SECURITY LABEL%';
SELECT 1 FROM citus_add_node('localhost', :worker_2_port);
NOTICE: issuing SELECT worker_create_or_alter_role('user1', 'CREATE ROLE user1 NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''', 'ALTER ROLE user1 NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''');SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified'
NOTICE: issuing SELECT worker_create_or_alter_role('user1', 'CREATE ROLE user1 NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''', 'ALTER ROLE user1 NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''');SECURITY LABEL FOR citus_tests_label_provider ON ROLE user1 IS 'citus unclassified'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
NOTICE: issuing SELECT worker_create_or_alter_role('user 2', 'CREATE ROLE "user 2" NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''', 'ALTER ROLE "user 2" NOSUPERUSER NOCREATEDB NOCREATEROLE INHERIT NOLOGIN NOREPLICATION NOBYPASSRLS CONNECTION LIMIT -1 PASSWORD NULL VALID UNTIL ''infinity''');SECURITY LABEL FOR citus_tests_label_provider ON ROLE "user 2" IS 'citus classified'
DETAIL: on server postgres@localhost:xxxxx connectionId: xxxxxxx
?column?
---------------------------------------------------------------------
1
(1 row)
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1');
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus_unclassified", "objtype": "role"}
worker_1 | {"label": "citus_unclassified", "objtype": "role"}
worker_2 | {"label": "citus_unclassified", "objtype": "role"}
coordinator | {"label": "citus unclassified", "objtype": "role"}
worker_1 | {"label": "citus unclassified", "objtype": "role"}
worker_2 | {"label": "citus unclassified", "objtype": "role"}
(3 rows)
SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type;
node_type | result
---------------------------------------------------------------------
coordinator | {"label": "citus classified", "objtype": "role"}
worker_1 | {"label": "citus classified", "objtype": "role"}
worker_2 | {"label": "citus classified", "objtype": "role"}
(3 rows)
-- cleanup
RESET citus.log_remote_commands;
DROP ROLE user1;
DROP ROLE user1, "user 2";

View File

@ -50,7 +50,6 @@ sub Usage()
print " --connection-timeout Timeout for connecting to worker nodes\n";
print " --mitmproxy Start a mitmproxy for one of the workers\n";
print " --worker-count Number of workers in Citus cluster (default: 2)\n";
print " --seclabel-test This test runs seclabel propagation tests";
exit 1;
}
@ -87,7 +86,6 @@ my $conninfo = "";
my $publicWorker1Host = "localhost";
my $publicWorker2Host = "localhost";
my $workerCount = 2;
my $seclabelTest = 0;
my $serversAreShutdown = "TRUE";
my $usingWindows = 0;
@ -122,7 +120,6 @@ GetOptions(
'worker-1-public-hostname=s' => \$publicWorker1Host,
'worker-2-public-hostname=s' => \$publicWorker2Host,
'worker-count=i' => \$workerCount,
'seclabel-test' => \$seclabelTest,
'help' => sub { Usage() });
my $fixopen = "$bindir/postgres.fixopen";
@ -495,6 +492,9 @@ push(@pgOptions, "citus.enable_change_data_capture=on");
push(@pgOptions, "citus.stat_tenants_limit = 2");
push(@pgOptions, "citus.stat_tenants_track = 'ALL'");
# We currently need this config for isolation tests and security label tests
push(@pgOptions, "citus.running_under_citus_test_suite=true");
# Some tests look at shards in pg_class, make sure we can usually see them:
push(@pgOptions, "citus.show_shards_for_app_name_prefixes='pg_regress'");
@ -563,7 +563,6 @@ if($isolationtester)
push(@pgOptions, "citus.metadata_sync_interval=1000");
push(@pgOptions, "citus.metadata_sync_retry_interval=100");
push(@pgOptions, "client_min_messages='warning'"); # pg12 introduced notice showing during isolation tests
push(@pgOptions, "citus.running_under_citus_test_suite=true");
# Disable all features of the maintenance daemon. Otherwise queries might
# randomly show temporarily as "waiting..." because they are waiting for the
@ -576,14 +575,6 @@ if($isolationtester)
push(@pgOptions, "citus.background_task_queue_interval=-1");
}
# if the security label propagation tests are running in the suite
# we need to load the label provider in PG_init by setting
# running_under_citus_test_suite GUC to true
if($seclabelTest)
{
push(@pgOptions, "citus.running_under_citus_test_suite=true");
}
# Add externally added options last, so they overwrite the default ones above
for my $option (@userPgOptions)
{

View File

@ -8,48 +8,79 @@
-- citus_add_node later
SELECT citus_remove_node('localhost', :worker_2_port);
-- create two roles, one with characters that need escaping
CREATE ROLE user1;
CREATE ROLE "user 2";
-- check an invalid label for our current dummy hook citus_test_object_relabel
SECURITY LABEL ON ROLE user1 IS 'invalid_label';
-- if we disable metadata_sync, the command will not be propagated
SET citus.enable_metadata_sync TO off;
SECURITY LABEL ON ROLE user1 IS 'citus_unclassified';
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1');
SECURITY LABEL 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');
SECURITY LABEL ON TABLE a IS 'citus_classified';
SELECT node_type, result FROM get_citus_tests_label_provider_labels('a');
DROP TABLE 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%';
-- then we run a security label statement which will use the same connection to the worker node
-- it should finish successfully
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_classified';
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus classified';
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS NULL;
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus_unclassified';
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus unclassified';
SECURITY LABEL for citus_tests_label_provider ON ROLE "user 2" IS 'citus classified';
\c - - - :worker_1_port
-- command not allowed from worker node
SECURITY LABEL for citus_tests_label_provider ON ROLE user1 IS 'citus unclassified';
\c - - - :master_port
RESET citus.log_remote_commands;
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1');
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type;
SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type;
-- adding a new node will fail because the label provider is not there
-- however, this is enough for testing as we can see that the SECURITY LABEL commands
-- will be propagated when adding a new node
-- add a new node and check that it also propagates the SECURITY LABEL statement to the new node
SET citus.log_remote_commands TO on;
SET citus.grep_remote_commands = '%SECURITY LABEL%';
SELECT 1 FROM citus_add_node('localhost', :worker_2_port);
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1');
SELECT node_type, result FROM get_citus_tests_label_provider_labels('user1') ORDER BY node_type;
SELECT node_type, result FROM get_citus_tests_label_provider_labels('"user 2"') ORDER BY node_type;
-- cleanup
RESET citus.log_remote_commands;
DROP ROLE user1;
DROP ROLE user1, "user 2";