citus/src/backend/distributed/sql/udfs
Halil Ozan Akgül b877d606c7
Adds 2PC distributed commands from other databases (#7203)
DESCRIPTION: Adds support for 2PC from non-Citus main databases

This PR only adds support for `CREATE USER` queries, other queries need
to be added. But it should be simple because this PR creates the
underlying structure.

Citus main database is the database where the Citus extension is
created. A non-main database is all the other databases that are in the
same node with a Citus main database.

When a `CREATE USER` query is run on a non-main database we:

1. Run `start_management_transaction` on the main database. This
function saves the outer transaction's xid (the non-main database
query's transaction id) and marks the current query as main db command.
2. Run `execute_command_on_remote_nodes_as_user("CREATE USER
<username>", <username to run the command>)` on the main database. This
function creates the users in the rest of the cluster by running the
query on the other nodes. The user on the current node is created by the
query on the outer, non-main db, query to make sure consequent commands
in the same transaction can see this user.
3. Run `mark_object_distributed` on the main database. This function
adds the user to `pg_dist_object` in all of the nodes, including the
current one.

This PR also implements transaction recovery for the queries from
non-main databases.
2023-12-22 19:19:41 +03:00
..
alter_distributed_table Adds alter_distributed_table and alter_table_set_access_method UDFs 2021-01-13 16:02:39 +03:00
alter_old_partitions_set_access_method Prevent C-style comments in all directories (#5250) 2021-09-09 11:54:58 +03:00
alter_role_if_exists Strip trailing whitespace and add final newline (#3186) 2019-11-21 14:25:37 +01:00
alter_table_set_access_method Adds alter_distributed_table and alter_table_set_access_method UDFs 2021-01-13 16:02:39 +03:00
any_value PG16 compatibility: ruleutils and successful CREATE EXTENSION (#7087) 2023-08-02 16:04:51 +03:00
citus_activate_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_add_inactive_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_add_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_add_rebalance_strategy Take improvement_threshold into the account in citus_add_rebalance_strategy() (#7247) 2023-10-09 13:13:08 +03:00
citus_add_secondary_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_backend_gpid Add citus_backend_gpid() 2022-03-03 15:29:40 +01:00
citus_blocking_pids Use global pids in citus_lock_waits 2022-02-21 17:46:34 +01:00
citus_calculate_gpid Add citus_backend_gpid() 2022-03-03 15:29:40 +01:00
citus_check_cluster_node_health Introduce citus_check_cluster_node_health UDF 2021-12-15 01:41:51 +03:00
citus_check_connection_to_node Introduce UDF to check worker connectivity 2021-12-03 02:30:28 +03:00
citus_cleanup_orphaned_resources 'Deferred Drop' and robust 'Shard Cleanup' for Splits. (#6258) 2022-09-06 12:11:20 -07:00
citus_cleanup_orphaned_shards Add citus_cleanup_orphaned_shards UDF 2021-06-04 11:23:07 +02:00
citus_conninfo_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_coordinator_nodeid Introduces citus_coordinator_node_id 2022-03-22 10:34:22 +03:00
citus_copy_shard_placement Introduce citus_copy_shard_placement UDF with nodeid 2023-01-12 16:57:51 +03:00
citus_disable_node Do not warn unncessarily when a node is removed 2022-05-18 17:23:38 +02:00
citus_dist_local_group_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_dist_node_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_dist_object_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_dist_partition_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_dist_placement_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_dist_shard_cache_invalidate Recreate invalidation functions for Citus10 2021-01-13 23:18:07 +03:00
citus_dist_stat_activity Updates citus_dist_stat_activity to use citus_stat_activity 2022-03-04 17:28:17 +03:00
citus_drain_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_drop_trigger Rename pg_dist tenant_schema to pg_dist_schema (#7001) 2023-06-14 12:12:15 +03:00
citus_extradata_container Remove migration paths to 9.3-1, introduce 9.3-2 2020-04-03 12:50:45 +03:00
citus_finalize_upgrade_to_citus11 Fixes a bug that prevents upgrades when there are no worker nodes 2022-06-28 15:54:49 +02:00
citus_finish_citus_upgrade Introduce a citus_finish_citus_upgrade() function 2022-06-13 13:15:15 +02:00
citus_finish_pg_upgrade PG16 compatibility - varnullingrels additions (#7107) 2023-08-15 13:07:55 +03:00
citus_get_node_clock Address the issues/comments from the original PR# 6315 2022-12-05 11:06:21 -08:00
citus_get_transaction_clock Address the issues/comments from the original PR# 6315 2022-12-05 11:06:21 -08:00
citus_internal_add_colocation_metadata Synchronize pg_dist_colocation metadata 2022-03-03 11:01:59 +01:00
citus_internal_add_object_metadata (1) Functions will be delegated even when present in the scope of an explicit 2022-01-19 16:43:33 -08:00
citus_internal_add_partition_metadata Use current user to sync metadata 2021-07-16 13:25:27 +02:00
citus_internal_add_placement_metadata Remove shardstate from placement insert functions (#6615) 2023-01-18 09:52:38 +01:00
citus_internal_add_shard_metadata Use current user to sync metadata 2021-07-16 13:25:27 +02:00
citus_internal_add_tenant_schema Rename pg_dist tenant_schema to pg_dist_schema (#7001) 2023-06-14 12:12:15 +03:00
citus_internal_adjust_local_clock_to_remote This implements a new UDF citus_get_cluster_clock() that returns a monotonically 2022-10-28 10:15:08 -07:00
citus_internal_database_command Adds create / drop database propagation support (#7240) 2023-11-21 16:43:51 +03:00
citus_internal_delete_colocation_metadata Synchronize pg_dist_colocation metadata 2022-03-03 11:01:59 +01:00
citus_internal_delete_partition_metadata Add non-blocking variant of create_distributed_table (#6087) 2022-08-30 15:35:40 +03:00
citus_internal_delete_placement_metadata Not undistribute Citus local table when converting it to a reference table / single-shard table 2023-08-29 12:57:28 +03:00
citus_internal_delete_shard_metadata Introduce citus_internal_delete_shard_metadata 2021-07-19 13:25:05 +02:00
citus_internal_delete_tenant_schema Rename pg_dist tenant_schema to pg_dist_schema (#7001) 2023-06-14 12:12:15 +03:00
citus_internal_global_blocked_processes Use global pids in citus_lock_waits 2022-02-21 17:46:34 +01:00
citus_internal_local_blocked_processes Use global pids in citus_lock_waits 2022-02-21 17:46:34 +01:00
citus_internal_mark_node_not_synced PR #6728  / commit - 10 2023-03-30 11:06:16 +03:00
citus_internal_unregister_tenant_schema_globally Add support for schema-based-sharding via a GUC (#6866) 2023-05-26 10:49:58 +03:00
citus_internal_update_none_dist_table_metadata Not undistribute Citus local table when converting it to a reference table / single-shard table 2023-08-29 12:57:28 +03:00
citus_internal_update_placement_metadata Use current user to sync metadata 2021-07-16 13:25:27 +02:00
citus_internal_update_relation_colocation Introduce citus_internal_update_relation_colocation 2021-08-03 11:44:58 +02:00
citus_is_clock_after This implements a new UDF citus_get_cluster_clock() that returns a monotonically 2022-10-28 10:15:08 -07:00
citus_is_coordinator Add a citus_is_coordinator function 2022-05-13 10:02:52 +02:00
citus_isolation_test_session_is_blocked Reduce isolation flakyness by improving blocked process detection (#6405) 2022-10-12 16:35:09 +02:00
citus_job_cancel Implement infrastructure to run sql jobs in the background (#6296) 2022-09-09 16:11:19 +03:00
citus_job_list Rebalance Progress Reporting API (#6576) 2023-01-16 16:17:31 +03:00
citus_job_status Improve rebalance reporting for retried tasks (#6683) 2023-01-31 15:26:52 +03:00
citus_job_wait Implement infrastructure to run sql jobs in the background (#6296) 2022-09-09 16:11:19 +03:00
citus_local_disk_space_stats When moving a shard to a new node ensure there is enough space (#4929) 2021-05-06 17:28:02 +03:00
citus_lock_waits Improve citus_lock_waits 2022-03-07 11:10:44 +01:00
citus_locks Introduce citus_locks view 2022-07-21 03:06:57 +03:00
citus_move_shard_placement Introduce citus_move_shard_placement UDF with nodeid 2023-01-12 16:57:51 +03:00
citus_node_capacity_1 Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
citus_nodeid_for_gpid Global PID Helper Functions (#5768) 2022-03-09 13:15:59 +03:00
citus_nodename_for_nodeid Global PID Helper Functions (#5768) 2022-03-09 13:15:59 +03:00
citus_nodeport_for_nodeid Global PID Helper Functions (#5768) 2022-03-09 13:15:59 +03:00
citus_pause_node_within_txn Adds citus_pause_node udf (#7089) 2023-09-01 11:39:30 +03:00
citus_pid_for_gpid Global PID Helper Functions (#5768) 2022-03-09 13:15:59 +03:00
citus_prepare_pg_upgrade PG16 compatibility - varnullingrels additions (#7107) 2023-08-15 13:07:55 +03:00
citus_rebalance_start Feature: run rebalancer in the background (#6215) 2022-09-12 20:46:53 +03:00
citus_rebalance_status Rebalance Progress Reporting API (#6576) 2023-01-16 16:17:31 +03:00
citus_rebalance_stop Feature: run rebalancer in the background (#6215) 2022-09-12 20:46:53 +03:00
citus_rebalance_wait Feature: run rebalancer in the background (#6215) 2022-09-12 20:46:53 +03:00
citus_remote_connection_stats Throttle connections to the worker nodes 2020-04-14 10:27:48 +02:00
citus_remove_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_run_local_command Avoid round trips while fixing index names (#5549) 2021-12-27 10:29:37 +01:00
citus_schema_distribute Add citus_schema_distribute/undistribute udfs to convert a schema into a tenant schema / back to a regular schema (#6933) 2023-06-12 18:41:31 +03:00
citus_schema_move Add citus_schema_move() function (#7180) 2023-09-08 12:03:53 +03:00
citus_schema_undistribute Add citus_schema_distribute/undistribute udfs to convert a schema into a tenant schema / back to a regular schema (#6933) 2023-06-12 18:41:31 +03:00
citus_schemas Add citus_schemas view (#6979) 2023-06-16 14:21:58 +03:00
citus_set_coordinator_host Add citus_set_coordinator_host 2021-01-08 13:36:26 +01:00
citus_set_default_rebalance_strategy Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
citus_set_node_property Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_shard_allowed_on_node_true Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
citus_shard_cost_1 Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
citus_shard_cost_by_disk_size Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
citus_shard_indexes_on_worker Convert citus.hide_shards_from_app_name_prefixes to citus.show_shards_for_app_name_prefixes 2022-05-03 14:22:13 +02:00
citus_shard_sizes Move SQL file changes for citus_shard_sizes fixes into the new 11.3-2 version (#7050) 2023-07-14 17:19:54 +03:00
citus_shards Move SQL file changes for citus_shard_sizes fixes into the new 11.3-2 version (#7050) 2023-07-14 17:19:54 +03:00
citus_shards_on_worker Convert citus.hide_shards_from_app_name_prefixes to citus.show_shards_for_app_name_prefixes 2022-05-03 14:22:13 +02:00
citus_split_shard_by_split_points default mode for shard splitting is set to auto. (#6179) 2022-08-17 12:18:47 +03:00
citus_stat_activity Introduces citus_stat_activity view 2022-03-03 16:19:20 +03:00
citus_stat_tenants Add CPU usage to citus_stat_tenants (#6844) 2023-04-12 16:23:00 +03:00
citus_stat_tenants_local Rename pg_dist tenant_schema to pg_dist_schema (#7001) 2023-06-14 12:12:15 +03:00
citus_stat_tenants_local_reset Multi tenant monitoring (#6725) 2023-04-05 17:44:17 +03:00
citus_stat_tenants_reset Multi tenant monitoring (#6725) 2023-04-05 17:44:17 +03:00
citus_tables Move SQL file changes for citus_shard_sizes fixes into the new 11.3-2 version (#7050) 2023-07-14 17:19:54 +03:00
citus_task_wait add citus_task_wait udf to wait on desired task status (#6475) 2022-12-12 22:41:03 +03:00
citus_total_relation_size Harden citus_tables against node failure 2020-12-13 15:10:40 +01:00
citus_unmark_object_distributed Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_update_node Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_update_shard_statistics Remove the word 'master' from Citus UDFs (#4472) 2021-01-13 12:10:43 +03:00
citus_update_table_statistics Fix upgrade and downgrade paths for master/citus_update_table_statistics (#4805) 2021-03-11 14:52:40 +03:00
citus_validate_rebalance_strategy_functions Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
commit_management_command_2pc Adds 2PC distributed commands from other databases (#7203) 2023-12-22 19:19:41 +03:00
create_citus_local_table Get rid of the sql dir for new udf 2021-01-27 15:52:37 +03:00
create_distributed_function (1) Functions will be delegated even when present in the scope of an explicit 2022-01-19 16:43:33 -08:00
create_distributed_table Adds shard_count parameter to create_distributed_table 2021-03-29 16:22:49 +03:00
create_distributed_table_concurrently Add non-blocking variant of create_distributed_table (#6087) 2022-08-30 15:35:40 +03:00
create_time_partitions Fix typos. Spurred spotting "connectios" in logs 2021-10-25 13:54:09 +00:00
distributed_tables_colocated Revoke usage from the citus schema 2019-10-23 00:08:17 +02:00
drop_old_time_partitions Support custom cast from / to timestamptz in time partition management UDFs (#6923) 2023-06-19 17:49:05 +03:00
execute_command_on_remote_nodes_as_user Adds 2PC distributed commands from other databases (#7203) 2023-12-22 19:19:41 +03:00
fetch_intermediate_results Implement fetch_intermediate_results 2019-12-18 10:46:35 -08:00
fix_all_partition_shard_index_names Update migration paths from 10.2 to 11.0 (#5459) 2021-11-11 13:55:28 +03:00
fix_partition_shard_index_names Update migration paths from 10.2 to 11.0 (#5459) 2021-11-11 13:55:28 +03:00
fix_pre_citus10_partitioned_table_constraint_names Introduce UDFs for fixing partitioned table constraint names 2021-01-29 17:32:20 +03:00
get_all_active_transactions Removes incorrect parameter from get_all_active_transactions 2022-07-06 11:35:46 +03:00
get_global_active_transactions Introduce global PID 2022-02-08 16:49:38 +03:00
get_missing_time_partition_ranges Support custom cast from / to timestamptz in time partition management UDFs (#6923) 2023-06-19 17:49:05 +03:00
get_rebalance_progress Add status column to get_rebalance_progress() (#6403) 2022-10-17 16:55:31 +03:00
get_rebalance_table_shards_plan Implement an improvement threshold in the rebalancer (#4927) 2021-05-11 14:24:59 +02:00
isolate_tenant_to_new_shard Nonblocking tenant isolation is supported by using split api. (#6167) 2022-08-17 11:13:07 +03:00
mark_object_distributed Adds 2PC distributed commands from other databases (#7203) 2023-12-22 19:19:41 +03:00
master_add_inactive_node Replicate reference tables to coordinator. 2019-11-15 05:50:19 -08:00
master_add_node Replicate reference tables to coordinator. 2019-11-15 05:50:19 -08:00
master_drain_node Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
master_set_node_property Add shouldhaveshards to pg_dist_node (#2960) 2019-10-22 16:47:16 +02:00
pg_cancel_backend Don't override postgres C symbols with our own (#6300) 2022-09-07 11:27:05 +02:00
pg_dist_rebalance_strategy_trigger_func Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
pg_dist_shard_placement_trigger_func Change SQL migration build process for easier reviews (#2951) 2019-09-13 18:44:27 +02:00
pg_terminate_backend Don't override postgres C symbols with our own (#6300) 2022-09-07 11:27:05 +02:00
read_intermediate_results Implement read_intermediate_results 2019-12-17 13:51:16 -08:00
rebalance_table_shards Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
remove_local_tables_from_metadata Automatically undistribute citus local tables when no more fkeys with reference tables (#4538) 2021-01-22 18:15:41 +03:00
repl_origin_helper CDC implementation for Citus using Logical Replication (#6623) 2023-03-28 16:00:21 +05:30
replicate_reference_tables Allow specifying the shard_transfer_mode when replicating reference tables (#6070) 2022-08-09 13:21:31 +03:00
replicate_table_shards Change SQL migration build process for easier reviews (#2951) 2019-09-13 18:44:27 +02:00
run_command_on_all_nodes Introduces citus_stat_activity view 2022-03-03 16:19:20 +03:00
run_command_on_coordinator Add a run_command_on_coordinator function 2022-05-19 10:26:09 +02:00
start_management_transaction Adds 2PC distributed commands from other databases (#7203) 2023-12-22 19:19:41 +03:00
start_metadata_sync_to_all_nodes Parallelize metadata syncing on node activate 2022-05-23 09:15:48 +02:00
stop_metadata_sync_to_node Add parameter to cleanup metadata (#5055) 2021-07-01 16:23:53 +03:00
time_partition_range Add a view for simple (time) partitions and their access methods 2021-01-08 11:28:15 +01:00
time_partitions Rewrite time_partitions join clause to avoid smallint[] operator 2021-02-18 12:01:18 +01:00
truncate_local_data_after_distributing_table Introduce truncate_local_data_after_distributing_table() 2020-04-17 13:21:34 +03:00
undistribute_table Add cascade option to undistribute_table 2021-01-07 15:41:49 +03:00
update_distributed_table_colocation Remove migration paths to 9.3-1, introduce 9.3-2 2020-04-03 12:50:45 +03:00
upgrade_to_reference_table Remove upgrade_to_reference_table UDF 2020-12-23 00:40:14 +01:00
worker_adjust_identity_column_seq_ranges Add upgrade/downgrade paths between v11.2.2 and v11.3.1 (#6820) 2023-04-06 12:46:09 +03:00
worker_change_sequence_dependency Adds alter_distributed_table and alter_table_set_access_method UDFs 2021-01-13 16:02:39 +03:00
worker_copy_table_to_node Use shard split copy code for blocking shard moves (#6098) 2022-08-01 20:10:36 +03:00
worker_create_or_alter_role Refactor alter role to work on distributed roles (#3739) 2020-04-16 12:23:27 +02:00
worker_create_or_replace_object Fix some typos in comments 2022-02-24 19:48:52 +03:00
worker_create_schema Use current user for repartition join temp schemas. 2020-02-04 09:48:20 -08:00
worker_drop_all_shell_tables PR #6728  / commit - 3 2023-03-30 10:53:20 +03:00
worker_drop_sequence_dependency Do not drop sequences when dropping metadata (#5584) 2022-01-06 09:48:34 +01:00
worker_drop_shell_table Handle tables and objects as metadata. Update UDFs accordingly 2022-01-31 16:20:15 +03:00
worker_fix_partition_shard_index_names Update migration paths from 10.2 to 11.0 (#5459) 2021-11-11 13:55:28 +03:00
worker_fix_pre_citus10_partitioned_table_constraint_names Introduce UDFs for fixing partitioned table constraint names 2021-01-29 17:32:20 +03:00
worker_last_saved_explain_analyze Include execution duration in worker_last_saved_explain_analyze 2020-06-11 02:54:54 -07:00
worker_nextval Fixes bug about int and smallint sequences on MX (#5254) 2021-09-09 23:41:07 +03:00
worker_partition_query_result Use intermediate results for re-partition joins 2022-02-23 19:40:21 +01:00
worker_partitioned_relation_size Fix worker partitioned size functions (#4922) 2021-04-26 10:29:46 +03:00
worker_partitioned_relation_total_size Fix worker partitioned size functions (#4922) 2021-04-26 10:29:46 +03:00
worker_partitioned_table_size Fix worker partitioned size functions (#4922) 2021-04-26 10:29:46 +03:00
worker_record_sequence_dependency Create sequence dependency during metadata syncing 2020-10-06 10:57:39 +02:00
worker_repartition_cleanup Add adaptive executor support for repartition joins (#3169) 2019-12-17 19:09:45 +03:00
worker_save_query_explain_analyze Implement EXPLAIN ANALYZE udfs. 2020-06-09 10:02:05 -07:00
worker_split_copy Add non-blocking variant of create_distributed_table (#6087) 2022-08-30 15:35:40 +03:00
worker_split_shard_release_dsm Add worker_split_shard_release_dsm udf to release dynamic shared memory (#6248) 2022-08-26 18:27:32 +05:30
worker_split_shard_replication_setup Unique names for replication artifacts (#6529) 2022-12-06 15:48:16 +03:00