Commit Graph

104 Commits (1c09469dd2f1bd2bf60ba3b049c54ad5c2b20c90)

Author SHA1 Message Date
German Eichberger 1c09469dd2
Adds a method to determine if current node is primary (#7720)
DESCRIPTION: Adds citus_is_primary_node() UDF to determine if the
current node is a primary node in the cluster.

---------

Co-authored-by: German Eichberger <geeichbe@microsoft.com>
Co-authored-by: Onur Tirtir <onurcantirtir@gmail.com>
2025-03-18 15:12:42 +00:00
Naisila Puka 6bd3474804 Rename foreach_ macros to foreach_declared_ macros (#7700)
This is prep work for successful compilation with PG17

PG17added foreach_ptr, foreach_int and foreach_oid macros
Relevant PG commit
14dd0f27d7cd56ffae9ecdbe324965073d01a9ff

14dd0f27d7

We already have these macros, but they are different with the
PG17 ones because our macros take a DECLARED variable, whereas
the PG16 macros declare a locally-scoped loop variable themselves.

Hence I am renaming our macros to foreach_declared_

I am separating this into its own PR since it touches many files. The
main compilation PR is https://github.com/citusdata/citus/pull/7699
2025-03-12 11:01:49 +03:00
Mehmet YILMAZ 4775715691
Fix race condition in citus_set_coordinator_host when adding multiple coordinator nodes concurrently (#7682)
When multiple sessions concurrently attempt to add the same coordinator
node using `citus_set_coordinator_host`, there is a potential race
condition. Both sessions may pass the initial metadata check
(`isCoordinatorInMetadata`), but only one will succeed in adding the
node. The other session will fail with an assertion error
(`Assert(!nodeAlreadyExists)`), causing the server to crash. Even though
the `AddNodeMetadata` function takes an exclusive lock, it appears that
the lock is not preventing the race condition before the initial
metadata check.

- **Issue**: The current logic allows concurrent sessions to pass the
check for existing coordinators, leading to an attempt to insert
duplicate nodes, which triggers the assertion failure.

- **Impact**: This race condition leads to crashes during operations
that involve concurrent coordinator additions, as seen in
https://github.com/citusdata/citus/issues/7646.

**Test Plan:**

- Isolation Test Limitation: An isolation test was added to simulate
concurrent additions of the same coordinator node, but due to the
behavior of PostgreSQL locking mechanisms, the test does not trigger the
edge case. The lock applied within the function serializes the
operations, preventing the race condition from occurring in the
isolation test environment.
While the edge case is difficult to reproduce in an isolation test, the
fix addresses the core issue by ensuring concurrency control through
proper locking.

- Existing Tests: All existing tests related to node metadata and
coordinator management have been run to ensure that no regressions were
introduced.

**After the Fix:**

- Concurrent attempts to add the same coordinator node will be
serialized. One session will succeed in adding the node, while the
others will skip the operation without crashing the server.

Co-authored-by: Mehmet YILMAZ <mehmet.yilmaz@microsoft.com>
2024-09-09 17:09:56 +03:00
Karina 683e10ab69
Fix error in master_disable_node/citus_disable_node (#7492)
This fixes #7454: master_disable_node() has only two arguments, but
calls citus_disable_node() that tries to read three arguments

Co-authored-by: Karina Litskevich <litskevichkarina@gmail.com>
2024-02-21 11:35:27 +00:00
eaydingol f01c5f2593
Move remaining citus_internal functions (#7478)
Moves the following functions to the Citus internal schema: 

citus_internal_local_blocked_processes
citus_internal_global_blocked_processes
citus_internal_mark_node_not_synced
citus_internal_unregister_tenant_schema_globally
citus_internal_update_none_dist_table_metadata
citus_internal_update_placement_metadata
citus_internal_update_relation_colocation
citus_internal_start_replication_origin_tracking
citus_internal_stop_replication_origin_tracking
citus_internal_is_replication_origin_tracking_active


#7405

---------

Co-authored-by: Jelte Fennema-Nio <jelte.fennema@microsoft.com>
2024-02-07 16:58:17 +03:00
Halil Ozan Akgül 739c6d26df
Fix inserting to pg_dist_object for queries from other nodes (#7402)
Running a query from a Citus non-main database that inserts to
pg_dist_object requires a new connection to the main database itself.
This PR adds that connection to the main database.

---------

Co-authored-by: Jelte Fennema-Nio <github-tech@jeltef.nl>
2024-01-11 16:05:14 +03:00
Onur Tirtir 1d55debb98
Support CREATE / DROP database commands from any node (#7359)
DESCRIPTION: Adds support for issuing `CREATE`/`DROP` DATABASE commands
from worker nodes

With this commit, we allow issuing CREATE / DROP DATABASE commands from
worker nodes too.
As in #7278, this is not allowed when the coordinator is not added to
metadata because we don't ever sync metadata changes to coordinator
when adding coordinator to the metadata via
`SELECT citus_set_coordinator_host('<hostname>')`, or equivalently, via
`SELECT citus_add_node(<coordinator_node_name>, <coordinator_node_port>, 0)`.

We serialize database management commands by acquiring a Citus specific
advisory lock on the first primary worker node if there are any workers in the
cluster. As opposed to what we've done in https://github.com/citusdata/citus/pull/7278
for role management commands, we try to avoid from running into distributed deadlocks
as much as possible. This is because, while distributed deadlocks that can happen around
role management commands can be detected by Citus, this is not the case for database
management commands because most of them cannot be run inside in a transaction block.
In that case, Citus cannot even detect the distributed deadlock because the command is not
part of a distributed transaction at all, then the command execution might not return the
control back to the user for an indefinite amount of time.
2024-01-08 16:47:49 +00:00
Nils Dijk 0620c8f9a6
Sort includes (#7326)
This change adds a script to programatically group all includes in a
specific order. The script was used as a one time invocation to group
and sort all includes throught our formatted code. The grouping is as
follows:

 - System includes (eg. `#include<...>`)
 - Postgres.h (eg. `#include "postgres.h"`)
- Toplevel imports from postgres, not contained in a directory (eg.
`#include "miscadmin.h"`)
 - General postgres includes (eg . `#include "nodes/..."`)
- Toplevel citus includes, not contained in a directory (eg. `#include
"citus_verion.h"`)
 - Columnar includes (eg. `#include "columnar/..."`)
 - Distributed includes (eg. `#include "distributed/..."`)

Because it is quite hard to understand the difference between toplevel
citus includes and toplevel postgres includes it hardcodes the list of
toplevel citus includes. In the same manner it assumes anything not
prefixed with `columnar/` or `distributed/` as a postgres include.

The sorting/grouping is enforced by CI. Since we do so with our own
script there are not changes required in our uncrustify configuration.
2023-11-23 18:19:54 +01:00
Onur Tirtir 240313e286
Support role commands from any node (#7278)
DESCRIPTION: Adds support from issuing role management commands from worker nodes

It's unlikely to get into a distributed deadlock with role commands, we
don't care much about them at the moment.
There were several attempts to reduce the chances of a deadlock but we
didn't any of them merged into main branch yet, see:
#7325
#7016
#7009
2023-11-10 09:58:51 +00:00
Gürkan İndibay b8bded6454
Adds citus_pause_node udf (#7089)
DESCRIPTION: Presenting citus_pause_node UDF enabling pausing by
node_id.

citus_pause_node takes a node_id parameter and fetches all the shards in
that node and puts AccessExclusiveLock on all the shards inside that
node. With this lock, insert is disabled, until citus_pause_node
transaction is closed.

---------

Co-authored-by: Hanefi Onaldi <Hanefi.Onaldi@microsoft.com>
2023-09-01 11:39:30 +03:00
Naisila Puka 2ba3bffe1e
Random warning fixes (#6974)
Citus build with PG16 fails because of the following warnings:
 - using char* instead of Datum
 - using pointer instead of oid
 - candidate function for format attribute
 - remove old definition from PG11 compatibility 62bf571ced

This commit fixes the above.
2023-06-09 14:36:43 +03:00
Teja Mupparti e444dd4f3f MERGE: Support reference table as source with local table as target 2023-05-02 11:37:29 -07:00
aykut-bozkurt a20f7e1a55
fixes update propagation bug when `citus_set_coordinator_host` is called more than once (#6837)
DESCRIPTION: Fixes update propagation bug when
`citus_set_coordinator_host` is called more than once.

Fixes https://github.com/citusdata/citus/issues/6731.
2023-04-11 11:27:16 +03:00
aykutbozkurt f2f0ec9dda PR #6728  / commit - 12
Force activated bare connections to close at transaction end.
2023-03-30 11:06:16 +03:00
aykutbozkurt 35dbdae5a4 PR #6728  / commit - 11
Let AddNodeMetadata to use metadatasync api during node addition.
2023-03-30 11:06:16 +03:00
aykutbozkurt fe00b3263a PR #6728  / commit - 10
Do not acquire strict lock on separate transaction to localhost as we already take the lock before.
But make sure that caller has the ExclusiveLock.
2023-03-30 11:06:16 +03:00
aykutbozkurt f8fb20cc95 PR #6728  / commit - 7
Remove unused old metadata sync methods.
2023-03-30 10:53:22 +03:00
aykutbozkurt bc25ba51c3 PR #6728  / commit - 5
Let `ActivateNode` use new metadata sync api.
2023-03-30 10:53:22 +03:00
Onur Tirtir 20a5f3af2b
Replace CITUS_TABLE_WITH_NO_DIST_KEY checks with HasDistributionKey() (#6743)
Now that we will soon add another table type having DISTRIBUTE_BY_NONE
as distribution method and that we want the code to interpret such
tables mostly as distributed tables, let's make the definition of those
other two table types more strict by removing
CITUS_TABLE_WITH_NO_DIST_KEY
macro.

And instead, use HasDistributionKey() check in the places where the
logic applies to all table types that have / don't have a distribution
key. In future PRs, we might want to convert some of those
HasDistributionKey() checks if logic only applies to Citus local /
reference tables, not the others.

And adding HasDistributionKey() also allows us to consider having
DISTRIBUTE_BY_NONE as the distribution method as a "table attribute"
that can apply to distributed tables too, rather something that
determines the table type.
2023-03-10 13:55:52 +03:00
Onur Tirtir e3cf7ace7c
Stabilize single_node.sql and others that report illegal node removal (#6751)
See
https://app.circleci.com/pipelines/github/citusdata/citus/30859/workflows/223d61db-8c1d-4909-9aea-d8e470f0368b/jobs/1009243.
2023-03-08 15:25:36 +03:00
Marco Slot 64e3fee89b
Remove shardstate leftovers (#6627)
Remove ShardState enum and associated logic.

Co-authored-by: Marco Slot <marco.slot@gmail.com>
Co-authored-by: Ahmet Gedemenli <afgedemenli@gmail.com>
2023-01-19 11:43:58 +03:00
Marco Slot ad3407b5ff
Revert "Make the metadata syncing less resource invasive [Phase-1]" (#6618)
Co-authored-by: Marco Slot <marco.slot@gmail.com>
2023-01-13 13:56:55 +01:00
Önder Kalacı a1aa96b32c
Make the metadata syncing less resource invasive [Phase-1] (#6537) 2023-01-04 11:36:45 +01:00
Ahmet Gedemenli 235047670d
Drop SHARD_STATE_TO_DELETE (#6494)
DESCRIPTION: Drop `SHARD_STATE_TO_DELETE` and use the cleanup records
instead

Drops the shard state that is used to mark shards as orphaned. Now we
insert cleanup records into `pg_dist_cleanup` so "orphaned" shards will
be dropped either by maintenance daemon or internal cleanup calls. With
this PR, we make the "cleanup orphaned shards" functions to be no-op, as
they would not be needed anymore.

This PR includes some naming changes about placement functions. We don't
need functions that filter orphaned shards, as there will be no orphaned
shards anymore.

We will also be introducing a small script with this PR, for users with
orphaned shards. We'll basically delete the orphaned shard entries from
`pg_dist_placement` and insert cleanup records into `pg_dist_cleanup`
for each one of them, during Citus upgrade.

We also have a lot of flakiness fixes in this PR.

Co-authored-by: Jelte Fennema <github-tech@jeltef.nl>
2023-01-03 14:38:16 +03:00
Hanefi Onaldi d4394b2e2d
Fix spacing in multiline strings (#6533)
When using multiline strings, we occasionally forget to add a single
space at the end of the first line. When this line is concatenated with
the next one, the resulting string has a missing space.
2022-12-01 23:42:47 +03:00
Philip Dubé cf69fc3652 Grammar: it's to its
Includes an error message

& one case of its to it's

Also fix "to the to" typos
2022-11-28 20:43:44 +00:00
Marco Slot 666696c01c
Deprecate citus.replicate_reference_tables_on_activate, make it always off (#6474)
Co-authored-by: Marco Slot <marco.slot@gmail.com>
2022-11-04 16:21:10 +01:00
Alexander Kukushkin 402a30a2b7
Allow citus_update_node() to work with nodes from different clusters (#6466)
DESCRIPTION: Allow citus_update_node() to work with nodes from different clusters

citus_update_node(), citus_nodename_for_nodeid(), and citus_nodeport_for_nodeid() functions only checked for nodes in their own clusters and hence last two returned NULLs and the first one showed an error is the nodeId was from a different cluster.

Fixes https://github.com/citusdata/citus/issues/6433
2022-11-02 10:07:01 +01:00
Naisila Puka 317dda6af1
Use RelationGetPrimaryKeyIndex for citus catalog tables (#6262)
pg_dist_node and pg_dist_colocation have a primary key index, not a replica identity index.

Citus catalog tables are created in public schema, which has replica identity index by default 
as primary key index. Later the citus catalog tables are moved to pg_catalog schema.

During pg_upgrade, all tables are recreated, and given that pg_dist_colocation is found in
pg_catalog schema, it is recreated in that schema, and when it is recreated it doesn't
have a replica identity index, because catalog tables have no replica identity.

Further action:
Do we even need to acquire this lock on the primary key index?
Postgres doesn't acquire such locks on indexes before deleting catalog tuples.
Also, catalog tuples don't have replica identities by definition.
2022-09-01 11:56:31 +03:00
Marco Slot 6bb31c5d75
Add non-blocking variant of create_distributed_table (#6087)
Added create_distributed_table_concurrently which is nonblocking variant of create_distributed_table.

It bases on the split API which takes advantage of logical replication to support nonblocking split operations.

Co-authored-by: Marco Slot <marco.slot@gmail.com>
Co-authored-by: aykutbozkurt <aykut.bozkurt1995@gmail.com>
2022-08-30 15:35:40 +03:00
Onder Kalaci b2e9a5baf1 Make sure citus_is_coordinator works on read replicas 2022-07-13 14:11:18 +02:00
Onder Kalaci 7157152f6c Do not send metadata changes during add node if citus.enable_metadata_sync is set to false 2022-05-30 13:24:31 +02:00
Onder Kalaci 010a2a408e Avoid assertion failure on citus_add_node 2022-05-30 12:22:09 +02:00
Gledis Zeneli 27ddb4fc8e
Do not obtain AccessShareLock before actual lock (#5965)
Do not obtain AccessShareLock before acquiring the distributed locks.

Acquiring an AccessShareLock ensures that the relations which we are trying to get a distributed lock on will not be dropped in the time between when the LOCK command is issued and the LOCK commands are send to the worker. However, this also leads to distributed deadlocks in such scenarios:

```sql
-- for dist lock acquiring order coor, w1, w2

-- on w2
LOCK t1 IN ACCESS EXLUSIVE MODE;
-- acquire AccessShareLock locally on t1 to ensure it is not dropped while we get ready to distribute the lock

      -- concurrently on w1
      LOCK t1 IN ACCESS EXLUSIVE MODE;
      -- acquire AccessShareLock locally on t1 to ensure it is not dropped while we get ready to distribute the lock
      -- acquire dist lock on coor, w1, gets blocked on local AccessShareLock on w2

-- on w2 continuation of the execution above
-- starts to acquire dist locks and gets blocked on the coor by the lock acquired by w1

-- distributed deadlock

``` 

We opt for avoiding such deadlocks with the cost of the possibility of running into errors when the relations on which we are trying to acquire locks on get dropped.
2022-05-23 13:06:38 +03:00
Onder Kalaci dd02e1755f Parallelize metadata syncing on node activate
It is often useful to be able to sync the metadata in parallel
across nodes.

Also citus_finalize_upgrade_to_citus11() uses
start_metadata_sync_to_primary_nodes() after this commit.

Note that this commit does not parallelize all pieces of node
activation or metadata syncing. Instead, it tries to parallelize
potenially large parts of metadata, which is the objects and
distributed tables (in general Citus tables).

In the future, it would be nice to sync the reference tables
in parallel across nodes.

Create ~720 distributed tables / ~23450 shards
```SQL
-- declaratively partitioned table
CREATE TABLE github_events_looooooooooooooong_name (
  event_id bigint,
  event_type text,
  event_public boolean,
  repo_id bigint,
  payload jsonb,
  repo jsonb,
  actor jsonb,
  org jsonb,
  created_at timestamp
) PARTITION BY RANGE (created_at);

SELECT create_time_partitions(
  table_name         := 'github_events_looooooooooooooong_name',
  partition_interval := '1 day',
  end_at             := now() + '24 months'
);

CREATE INDEX ON github_events_looooooooooooooong_name USING btree (event_id, event_type, event_public, repo_id);
SELECT create_distributed_table('github_events_looooooooooooooong_name', 'repo_id');

SET client_min_messages TO ERROR;

```

across 1 node: almost same as expected
```SQL

SELECT start_metadata_sync_to_primary_nodes();
Time: 15664.418 ms (00:15.664)

select start_metadata_sync_to_node(nodename,nodeport) from pg_dist_node;
Time: 14284.069 ms (00:14.284)
```

across 7 nodes: ~3.5x improvement
```SQL

SELECT start_metadata_sync_to_primary_nodes();
┌──────────────────────────────────────┐
│ start_metadata_sync_to_primary_nodes │
├──────────────────────────────────────┤
│ t                                    │
└──────────────────────────────────────┘
(1 row)

Time: 25711.192 ms (00:25.711)

-- across 7 nodes
select start_metadata_sync_to_node(nodename,nodeport) from pg_dist_node;
Time: 82126.075 ms (01:22.126)
```
2022-05-23 09:15:48 +02:00
Onder Kalaci 127450466e Do not warn unncessarily when a node is removed
In the past (pre-11), we allowed removing worker nodes
that had active placements for replicated distributed
table, without even checking if there are any other
replicas of the same placement.

However, with #5469, we prevent disabling nodes via a hard
error when there is the last active placement of shard, as we
do for reference tables. Note that otherwise, we'd allow
users to lose data.

As of today, the NOTICE is completely irrelevant.
2022-05-18 17:23:38 +02:00
Onder Kalaci b4dbd84743 Prevent distributed queries while disabling first worker node
First worker node has a special meaning for modifications on the replicated tables

It is used to acquire a remote lock, such that the modifications are serialized.

With this commit, we make sure that we do not let any distributed query to see a
different 'first worker node' while first worker node is disabled.

Note that, maybe implicitly mentioned above, when first worker node is disabled,
the first worker node changes, that's why we have to handle the situation.
2022-05-18 17:21:12 +02:00
Onder Kalaci db998b3d66 Adds "sync" option to citus_disable_node() UDF
Before this commit, we had:
```SQL
SELECT citus_disable_node(nodename, nodeport, force boolean DEFAULT false)
```

Where, we allow forcing to disable first worker node with
`force:=true`. However, it entails the risk for losing
data / diverging placement data etc.

With `force` flag, we control disabling the first worker node,
and with `async` flag we control whether the changes are done
via bg worker or immediately.

```SQL
SELECT citus_disable_node(nodename, nodeport, force boolean DEFAULT false, sync boolean DEFAULT false)
```

Where we can achieve all the following:

| Mode  | Data loss possibility | Can run in 2PC | Handle multiple node failures | Immediately effective |
| --- |--- |--- |--- |--- |
| force:false, sync: false  | false   | true  | true  | false |
| force:false, sync: true   | false  | false | false | true |
| force:true, sync: false   | true   | true  | true   | false |
| force:true, sync: true    | false  | false | false  | true |
2022-05-18 17:21:12 +02:00
Marco Slot 6fad5dc207 Add a citus_is_coordinator function 2022-05-13 10:02:52 +02:00
Onder Kalaci 5fc7661169 Do not set coordinator's metadatasynced column to false
After a disable_node
2022-04-25 09:25:59 +02:00
Onder Kalaci b0b91bab04 Rename metadata sync to node metadata sync where applicable 2022-04-07 17:51:31 +02:00
Halil Ozan Akgul 4dbc760603 Introduces citus_coordinator_node_id 2022-03-22 10:34:22 +03:00
Gledis Zeneli 56ab64b747
Patches #5758 with some more error checks (#5804)
Add error checks to detect failed connection and don't ping secondary nodes to detect self reference.
2022-03-15 15:02:47 +03:00
Gledis Zeneli 2cb02bfb56
Fix node adding itself with citus_add_node leading to deadlock (Fix #5720) (#5758)
If a worker node is being added, a command is sent to get the server_id of the worker from the pg_dist_node_metadata table. If the worker's id is the same as the node executing the code, we will know the node is trying to add itself. If the node tries to add itself without specifying `groupid:=0` the operation will result in an error.
2022-03-10 17:46:33 +03:00
Halil Ozan Akgül 333bcc7948
Global PID Helper Functions (#5768)
* Introduces citus_nodename_for_nodeid and citus_nodeport_for_nodeid functions

* Introduces citus_nodeid_for_gpid and citus_pid_for_gpid functions

* Add tests
2022-03-09 13:15:59 +03:00
Marco Slot 3ba61244b8 Synchronize pg_dist_colocation metadata 2022-03-03 11:01:59 +01:00
Onder Kalaci df95d59e33 Drop support for CitusInitiatedBackend
CitusInitiatedBackend was a pre-mature implemenation of the whole
GlobalPID infrastructure. We used it to track whether any individual
query is triggered by Citus or not.

As of now, after GlobalPID is already in place, we don't need
CitusInitiatedBackend, in fact it could even be wrong.
2022-02-24 12:12:43 +01:00
Hanefi Onaldi f4e8af2c22
Do not acquire locks on node metadata explicitly 2022-02-24 03:19:56 +03:00
Halil Ozan Akgul f6cd4d0f07 Overrides pg_cancel_backend and pg_terminate_backend to accept global pid 2022-02-21 16:41:35 +03:00
Onder Kalaci ff234fbfd2 Unify old GUCs into a single one
Replaces citus.enable_object_propagation with citus.enable_metadata_sync

Also, within Citus 11 release cycle, we added citus.enable_metadata_sync_by_default,
that is also replaced with citus.enable_metadata_sync.

In essence, when citus.enable_metadata_sync is set to true, all the objects
and the metadata is send to the remote node.

We strongly advice that the users never changes the value of
this GUC.
2022-02-04 10:52:56 +01:00