Commit Graph

32 Commits (2a768176c4d5256fa75ccef83bffd46ec54a1745)

Author SHA1 Message Date
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
Gledis Zeneli 4c6f62efc6
Switch to using LOCK instead of lock_relation_if_exists in TRUNCATE (#5930)
Breaking down #5899 into smaller PR-s

This particular PR changes the way TRUNCATE acquires distributed locks on the relations it is truncating to use the LOCK command instead of lock_relation_if_exists. This has the benefit of using pg's recursive locking logic it implements for the LOCK command instead of us having to resolve relation dependencies and lock them explicitly. While this does not directly affect truncate, it will allow us to generalize this locking logic to then log different relations where the pg recursive locking will become useful (e.g. locking views).

This implementation is a bit more complex that it needs to be due to pg not supporting locking foreign tables. We can however, still lock foreign tables with lock_relation_if_exists. So for a command:

TRUNCATE dist_table_1, dist_table_2, foreign_table_1, foreign_table_2, dist_table_3;

We generate and send the following command to all the workers in metadata:
```sql
SEL citus.enable_ddl_propagation TO FALSE;
LOCK dist_table_1, dist_table_2 IN ACCESS EXCLUSIVE MODE;
SELECT lock_relation_if_exists('foreign_table_1', 'ACCESS EXCLUSIVE');
SELECT lock_relation_if_exists('foreign_table_2', 'ACCESS EXCLUSIVE');
LOCK dist_table_3 IN ACCESS EXCLUSIVE MODE;
SEL citus.enable_ddl_propagation TO TRUE;
```

Note that we need to alternate between the lock command and lock_table_if_exists in order to preserve the TRUNCATE order of relations.
When pg supports locking foreign tables, we will be able to massive simplify this logic and send a single LOCK command.
2022-05-11 18:38:48 +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
Burak Velioglu 8ae7577581
Use superuser connection while syncing dependent objects' pg_dist_object tuples 2022-02-07 17:50:45 +03:00
Onder Kalaci d405993b57 Make sure to use a dedicated metadata connection
With this commit, we make sure to use a dedicated connection per
node for all the metadata operations within the same transaction.

This is needed because the same metadata (e.g., metadata includes
the distributed table on the workers) can be modified accross
multiple connections.

With this connection we guarantee that there is a single metadata connection.
But note that this connection can be used for any other operation.
In other words, this connection is not only reserved for metadata
operations.
2021-11-26 14:36:28 +01:00
Onder Kalaci 5f02d18ef8 transactional metadata sync for maintanince daemon
As we use the current user to sync the metadata to the nodes
with #5105 (and many other PRs), there is no reason that
prevents us to use the coordinated transaction for metadata syncing.

This commit also renames few functions to reflect their actual
implementation.
2021-08-09 10:34:55 +02:00
Onder Kalaci 2c349e6dfd Use current user to sync metadata
Before this commit, we always synced the metadata with superuser.
However, that creates various edge cases such as visibility errors
or self distributed deadlocks or complicates user access checks.

Instead, with this commit, we use the current user to sync the metadata.
Note that, `start_metadata_sync_to_node` still requires super user
because accessing certain metadata (like pg_dist_node) always require
superuser (e.g., the current user should be a superuser).

However, metadata syncing operations regarding the distributed
tables can now be done with regular users, as long as the user
is the owner of the table. A table owner can still insert non-sense
metadata, however it'd only affect its own table. So, we cannot do
anything about that.
2021-07-16 13:25:27 +02:00
Jelte Fennema 2f29d4e53e Continue to remove shards after first failure in DropMarkedShards
The comment of DropMarkedShards described the behaviour that after a
failure we would continue trying to drop other shards. However the code
did not do this and would stop after the first failure. Instead of
simply fixing the comment I fixed the code, because the described
behaviour is more useful. Now a single shard that cannot be removed yet
does not block others from being removed.
2021-04-30 15:42:09 +03:00
Sait Talha Nisanci 510535f558 address feedback 2020-07-13 19:45:02 +03:00
Sait Talha Nisanci db1b78148c send schema creation/cleanup to coordinator in repartitions
We were using ALL_WORKERS TargetWorkerSet while sending temporary schema
creation and cleanup. We(well mostly I) thought that ALL_WORKERS would also include coordinator when it is added as a worker. It turns out that it was FILTERING OUT the coordinator even if it is added as a worker to the cluster.

So to have some context here, in repartitions, for each jobId we create
(at least we were supposed to) a schema in each worker node in the cluster. Then we partition each shard table into some intermediate files, which is called the PARTITION step. So after this partition step each node has some intermediate files having tuples in those nodes. Then we fetch the partition files to necessary worker nodes, which is called the FETCH step. Then from the files we create intermediate tables in the temporarily created schemas, which is called a MERGE step. Then after evaluating the result, we remove the temporary schemas(one for each job ID in each node) and files.

If node 1 has file1, and node 2 has file2 after PARTITION step, it is
enough to either move file1 from node1 to node2 or vice versa. So we
prune one of them.

In the MERGE step, if the schema for a given jobID doesn't exist, the
node tries to use the `public` schema if it is a superuser, which is
actually added for testing in the past.

So when we were not sending schema creation comands for each job ID to
the coordinator(because we were using ALL_WORKERS flag, and it doesn't
include the coordinator), we would basically not have any schemas for
repartitions in the coordinator. The PARTITION step would be executed on
the coordinator (because the tasks are generated in the planner part)
and it wouldn't give us any error because it doesn't have anything to do
with the temporary schemas(that we didn't create). But later two things
would happen:

- If by chance the fetch is pruned on the coordinator side, we the other
nodes would fetch the partitioned files from the coordinator and execute
the query as expected, because it has all the information.
- If the fetch tasks are not pruned in the coordinator, in the MERGE
step, the coordinator would either error out saying that the necessary
schema doesn't exist, or it would try to create the temporary tables
under public schema ( if it is a superuser). But then if we had the same
task ID with different jobID it would fail saying that the table already
exists, which is an error we were getting.

In the first case, the query would work okay, but it would still not do
the cleanup, hence we would leave the partitioned files from the
PARTITION step there. Hence ensure_no_intermediate_data_leak would fail.

To make things more explicit and prevent such bugs in the future,
ALL_WORKERS is named as ALL_NON_COORD_WORKERS. And a new flag to return
all the active nodes is added as ALL_DATA_NODES. For repartition case,
we don't use the only-reference table nodes but this version makes the
code simpler and there shouldn't be any significant performance issue
with that.
2020-07-13 19:20:15 +03:00
SaitTalhaNisanci 15290bc43b
remove unused worker methods (#4017) 2020-07-10 13:45:55 +03:00
SaitTalhaNisanci 3f50165365
rename TargetWorkerSet enums (#4015)
Rename TargetWorkerSet enums to make them more explicit about what they
mean. Ideally it would be good to treat everything as a node without the
'worker' concept because it makes things complicated. Another
improvement could be to rename TargetWorkerSet as TargetNodeSet but it
goes to renaming many occurrences of Worker, which is probably too big
for this PR.
2020-07-10 11:21:27 +03:00
Philip Dubé 20abc4d2b5
Replace foreach with foreach_ptr/foreach_oid (#3544) 2020-02-27 16:54:49 +01:00
SaitTalhaNisanci 82d22b34fe
create temp schemas in parallel (#3540) 2020-02-26 16:20:08 +03:00
SaitTalhaNisanci d94c3fd43d
send repartition cleanup jobs in parallel to all workers (#3485)
* send repartition cleanup jobs in parallel to all workers

* add review items
2020-02-26 13:44:06 +03:00
Philip Dubé 73c06fae3b Introduce GetDistributeObjectOps to organize dispatch of logic dependent on node/object type 2020-01-09 18:24:29 +00:00
SaitTalhaNisanci 7ff4ce2169
Add adaptive executor support for repartition joins (#3169)
* WIP

* wip

* add basic logic to run a single job with repartioning joins with adaptive executor

* fix some warnings and return in ExecuteDependedTasks if there is none

* Add the logic to run depended jobs in adaptive executor

The execution of depended tasks logic is changed. With the current
logic:
- All tasks are created from the top level task list.
- At one iteration:
	- CurTasks whose dependencies are executed are found.
	- CurTasks are executed in parallel with adapter executor main
logic.
- The iteration is repeated until all tasks are completed.

* Separate adaptive executor repartioning logic

* Remove duplicate parts

* cleanup directories and schemas

* add basic repartion tests for adaptive executor

* Use the first placement to fetch data

In task tracker, when there are replicas, we try to fetch from a replica
for which a map task is succeeded. TaskExecution is used for this,
however TaskExecution is not used in adaptive executor. So we cannot use
the same thing as task tracker.

Since adaptive executor fails when a map task fails (There is no retry
logic yet). We know that if we try to execute a fetch task, all of its
map tasks already succeeded, so we can just use the first one to fetch
from.

* fix clean directories logic

* do not change the search path while creating a udf

* Enable repartition joins with adaptive executor with only enable_reparitition_joins guc

* Add comments to adaptive_executor_repartition

* dont run adaptive executor repartition test in paralle with other tests

* execute cleanup only in the top level execution

* do cleanup only in the top level ezecution

* not begin a transaction if repartition query is used

* use new connections for repartititon specific queries

New connections are opened to send repartition specific queries. The
opened connections will be closed at the FinishDistributedExecution.

While sending repartition queries no transaction is begun so that
we can see all changes.

* error if a modification was done prior to repartition execution

* not start a transaction if a repartition query and sql task, and clean temporary files and schemas at each subplan level

* fix cleanup logic

* update tests

* add missing function comments

* add test for transaction with DDL before repartition query

* do not close repartition connections in adaptive executor

* rollback instead of commit in repartition join test

* use close connection instead of shutdown connection

* remove unnecesary connection list, ensure schema owner before removing directory

* rename ExecuteTaskListRepartition

* put fetch query string in planner not executor as we currently support only replication factor = 1 with adaptive executor and repartition query and we know the query string in the planner phase in that case

* split adaptive executor repartition to DAG execution logic and repartition logic

* apply review items

* apply review items

* use an enum for remote transaction state and fix cleanup for repartition

* add outside transaction flag to find connections that are unclaimed instead of always opening a new transaction

* fix style

* wip

* rename removejobdir to partition cleanup

* do not close connections at the end of repartition queries

* do repartition cleanup in pg catch

* apply review items

* decide whether to use transaction or not at execution creation

* rename isOutsideTransaction and add missing comment

* not error in pg catch while doing cleanup

* use replication factor of the creation time, not current time to decide if task tracker should be chosen

* apply review items

* apply review items

* apply review item
2019-12-17 19:09:45 +03:00
Hadi Moshayedi 2268a9cae6 Error for metadata commands if any metadata node is out-of-sync (#3226)
* Error for metadata commands if any metadata node is out-of-sync

* Make the functions have separate APIs for all workers/metadata workers
2019-11-27 09:52:57 +01:00
SaitTalhaNisanci 94a7e6475c
Remove copyright years (#2918)
* Update year as 2012-2019

* Remove copyright years
2019-10-15 17:44:30 +03:00
Onder Kalaci 3be72ce42f Make sure that distributed functions always have the correct user
Objectives:

(a) both super user and regular user should have the correct owner for the function on the worker
(b) The transactional semantics would work fine for both super user and regular user
(c) non-super-user and non-function owner would get a reasonable error message if tries to distribute the function

Co-authored-by: @serprex
2019-10-04 21:38:49 +00:00
Marco Slot e58d76c5f6 Fix assert failure in bare SELECT FROM reference table FOR UPDATE in MX 2019-09-23 17:00:09 +02:00
Philip Dubé 492d1b2cba ActivePrimaryNodeList: add lockMode parameter 2019-09-13 17:44:56 +00:00
Nils Dijk 2879689441
Distribute Types to worker nodes (#2893)
DESCRIPTION: Distribute Types to worker nodes

When to propagate
==============

There are two logical moments that types could be distributed to the worker nodes
 - When they get used ( just in time distribution )
 - When they get created ( proactive distribution )

The just in time distribution follows the model used by how schema's get created right before we are going to create a table in that schema, for types this would be when the table uses a type as its column.

The proactive distribution is suitable for situations where it is benificial to have the type on the worker nodes directly. They can later on be used in queries where an intermediate result gets created with a cast to this type.

Just in time creation is always the last resort, you cannot create a distributed table before the type gets created. A good example use case is; you have an existing postgres server that needs to scale out. By adding the citus extension, add some nodes to the cluster, and distribute the table. The type got created before citus existed. There was no moment where citus could have propagated the creation of a type.

Proactive is almost always a good option. Types are not resource intensive objects, there is no performance overhead of having 100's of types. If you want to use them in a query to represent an intermediate result (which happens in our test suite) they just work.

There is however a moment when proactive type distribution is not beneficial; in transactions where the type is used in a distributed table.

Lets assume the following transaction:

```sql
BEGIN;
CREATE TYPE tt1 AS (a int, b int);
CREATE TABLE t1 AS (a int PRIMARY KEY, b tt1);
SELECT create_distributed_table('t1', 'a');
\copy t1 FROM bigdata.csv
```

Types are node scoped objects; meaning the type exists once per worker. Shards however have best performance when they are created over their own connection. For the type to be visible on all connections it needs to be created and committed before we try to create the shards. Here the just in time situation is most beneficial and follows how we create schema's on the workers. Outside of a transaction block we will just use 1 connection to propagate the creation.

How propagation works
=================

Just in time
-----------

Just in time propagation hooks into the infrastructure introduced in #2882. It adds types as a supported object in `SupportedDependencyByCitus`. This will make sure that any object being distributed by citus that depends on types will now cascade into types. When types are depending them self on other objects they will get created first.

Creation later works by getting the ddl commands to create the object by its `ObjectAddress` in `GetDependencyCreateDDLCommands` which will dispatch types to `CreateTypeDDLCommandsIdempotent`.

For the correct walking of the graph we follow array types, when later asked for the ddl commands for array types we return `NIL` (empty list) which makes that the object will not be recorded as distributed, (its an internal type, dependant on the user type).

Proactive distribution
---------------------

When the user creates a type (composite or enum) we will have a hook running in `multi_ProcessUtility` after the command has been applied locally. Running after running locally makes that we already have an `ObjectAddress` for the type. This is required to mark the type as being distributed.

Keeping the type up to date
====================

For types that are recorded in `pg_dist_object` (eg. `IsObjectDistributed` returns true for the `ObjectAddress`) we will intercept the utility commands that alter the type.
 - `AlterTableStmt` with `relkind` set to `OBJECT_TYPE` encapsulate changes to the fields of a composite type.
 - `DropStmt` with removeType set to `OBJECT_TYPE` encapsulate `DROP TYPE`.
 - `AlterEnumStmt` encapsulates changes to enum values.
    Enum types can not be changed transactionally. When the execution on a worker fails a warning will be shown to the user the propagation was incomplete due to worker communication failure. An idempotent command is shown for the user to re-execute when the worker communication is fixed.

Keeping types up to date is done via the executor. Before the statement is executed locally we create a plan on how to apply it on the workers. This plan is executed after we have applied the statement locally.

All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables.
2019-09-13 17:46:07 +02:00
Onder Kalaci 76aa6951c2 Properly send commands to other nodes
We previously implemented OTHER_WORKERS_WITH_METADATA tag. However,
that was wrong. See the related discussion:
     https://github.com/citusdata/citus/issues/2320

Instead, we switched using OTHER_WORKER_NODES and make the command
that we're running optional such that even if the node is not a
metadata node, we won't be in trouble.
2018-09-10 16:01:30 +03:00
Onder Kalaci 26e308bf2a Support TRUNCATE from the MX worker nodes
This commit enables support for TRUNCATE on both
distributed table and reference tables.

The basic idea is to acquire lock on the relation by sending
the TRUNCATE command to all metedata worker nodes. We only
skip sending the TRUNCATE command to the node that actually
executus the command to prevent a self-distributed-deadlock.
2018-09-03 14:06:31 +03:00
Onder Kalaci 97ba7bf2eb Add the option to skip the node that is executing the node 2018-09-03 14:01:24 +03:00
velioglu bd30e3e908 Add support for writing to reference tables from MX nodes 2018-08-27 18:15:04 +03:00
Jason Petersen d904e96c59
Address MX CONCURRENTLY problems
Adds a non-transactional multi-command method to propagate DDLs to all
MX/metadata-synced nodes.
2017-04-03 11:19:15 -06:00
Jason Petersen 6f4886cd11
Revert "Remove unused SendCommandToWorker"
This reverts commit c8c308c109.
2017-03-13 15:48:51 -06:00
Brian Cloutier c8c308c109 Remove unused SendCommandToWorker 2017-03-08 16:30:23 +03:00
Eren Basak fb88b167a7 Propagate node add/remove to the nodes with hasmetadata=true
This change propagates the changes done by `master_add_node` and `master_remove_node`
to the workers that contain metadata.
2016-12-02 14:43:32 +03:00
Eren Basak cee7b54e7c Add worker transaction and transaction recovery infrastructure 2016-10-18 14:18:14 +03:00