DESCRIPTION: Add infrastructure to run long running management operations in background
This infrastructure introduces the primitives of jobs and tasks.
A task consists of a sql statement and an owner. Tasks belong to a
Job and can depend on other tasks from the same job.
When there are either runnable or running tasks we would like to
make sure a bacgrkound task queue monitor process is running. A Task
could be in running state while there is actually no monitor present
due to a database restart or failover. Once the monitor starts it
will reset any running task to its runnable state.
To make sure only one background task queue monitor is ever running
at once it will acquire an advisory lock that self conflicts.
Once a task is done it will find all tasks depending on this task.
After checking that the task doesn't have unmet dependencies it will
transition the task from blocked to runnable state for the task to
be picked up on a subsequent task start.
Currently only one task can be running at a time. This can be
improved upon in later releases without changes to the higher level
API.
The initial goal for this background tasks is to allow a rebalance
to run in the background. This will be implemented in a subsequent PR.
Previously we would create foreign keys to reference table in an extra
fast way at the end of a shard move. This uses that same logic to also
do it for foreign keys between distributed tables.
Fixes#6141
Introduces a new GUC named citus.skip_constraint_validation, which basically skips constraint validation when set to on.
For some several places that we hack to skip the foreign key validation phase, now we use this GUC.
When introducing our overrides of pg_cancel_backend and
pg_terminate_backend we accidentally did that in such a way that we
cannot call the original pg_cancel_backend and pg_terminate_backend from
C anymore. This happened because we defined the exact same symbols in
our shared library as postgres does in its own binary.
This fixes that by using a different names for the C function than for
the SQL function.
Making this work in all upgrade and downgrade scenarios is not trivial
though, because we actually need to remove the C function definition.
Postgres errors in two different times when the symbol that a C function
wants to call is not defined in the library it expects it in:
1. When creating the SQL function definition
2. When calling the SQL function
Item 1 causes an issue when creating our extension for the first time.
We then go execute all the migrations that we have. So if the 11.0
migration contains a SQL function definition that still references the
pg_cancel_backend symbol, that migration will fail. This issue is solved
by actually changing the SQL definition in the old migration.
This is not enough to fix all issues though. Item 2 causes an issue
after an upgrade to 11.1, because it won't have the new definition of
the SQL function. This is solved by recreating the SQL functions in the
migration to 11.1. That way it gets the new definition.
Then finally there's the case of downgrades. To continue to make our
pg_cancel_backend SQL function work after downgrading, we will need to
make a patch release for 11.0 that includes the new citus_cancel_backend
symbol. This is done in a separate commit.
DESCRIPTION:
This PR adds support for 'Deferred Drop' and robust 'Shard Cleanup' for Splits.
Common Infrastructure
This PR introduces new common infrastructure so as any operation that wants robust cleanup of resources can register with the cleaner and have the resources cleaned appropriately based on a specified policy. 'Shard Split' is the first consumer using this new infrastructure.
Note : We only support adding 'shards' as resources to be cleaned-up right now but the framework will be extended to support other resources in future.
Deferred Drop for Split
Deferred Drop Support ensures that shards undergoing split are not dropped inline as part of operation but dropped later when no active read queries are running on shard. This helps with :
Avoids any potential deadlock scenarios that can cause long running Split operation to rollback.
Avoids Split operation blocking writes and then getting blocked (due to running queries on the shard) when trying to drop shards.
Deferred drop is the new default behavior going forward.
Shard Cleaner Extension
Shard Cleaner is a background task responsible for deferred drops in case of 'Move' operations.
The cleaner has been extended to ensure robust cleanup of shards (dummy shards and split children) in case of a failure based on the new infrastructure mentioned above. The cleaner also handles deferred drop for 'Splits'.
TESTING:
New test ''citus_split_shard_by_split_points_deferred_drop' to test deferred drop support.
New test 'failure_split_cleanup' to test shard cleanup with failures in different stages.
Update 'isolation_blocking_shard_split and isolation_non_blocking_shard_split' for deferred drop.
Added non-deferred drop version of existing tests : 'citus_split_shard_no_deferred_drop' and 'citus_non_blocking_splits_no_deferred_drop'
* Fix issue : 6109 Segfault or (assertion failure) is possible when using a SQL function
* DESCRIPTION: Ensures disallowing the usage of SQL functions referencing to a distributed table and prevents a segfault.
Using a SQL function may result in segmentation fault in some cases.
This change fixes the issue by throwing an error message when a SQL function cannot be handled.
Fixes#6109.
* DESCRIPTION: Ensures disallowing the usage of SQL functions referencing to a distributed table and prevents a segfault.
Using a SQL function may result in segmentation fault in some cases. This change fixes the issue by throwing an error message when a SQL function cannot be handled.
Fixes#6109.
Co-authored-by: Emel Simsek <emel.simsek@microsoft.com>
PG15 allows numeric scale to be negative or greater than precision. This
causes issues and we may end up routing queries to a wrong shard due to
differing hash results after rounding.
Formerly, when specifying NUMERIC(precision, scale), the scale had to be
in the range [0, precision], which was per SQL spec. PG15 extends the
range of allowed scales to [-1000, 1000].
A negative scale implies rounding before the decimal point. For
example, a column might be declared with a scale of -3 to round values
to the nearest thousand. Note that the display scale remains
non-negative, so in this case the display scale will be zero, and all
digits before the decimal point will be displayed.
Relevant PG commit: 085f931f52494e1f304e35571924efa6fcdc2b44
Pre PG15, renaming child triggers on partitions is allowed. When
creating a trigger in a distributed parent partitioned table, the
triggers on the shards of the partitions have the same name with
the triggers on the corresponding parent shards of the parent
table. Therefore, they don't have the same appended shard id as
the shard id of the partition. Hence, when trying to rename a
child trigger on a partition of a distributed table, we can't
correctly find the triggers on the shards of the partition in
order to rename them since we append a different shard id to the
name of the trigger. Since we can't find the trigger we get a
misleading error of inexistent trigger.
In this commit we prohibit renaming child triggers on distributed
partitions altogether.
Sometimes in CI our isolation_citus_dist_activity test fails randomly
like this:
```diff
step s2-view-dist:
SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) AND backend_type = 'client backend' ORDER BY query DESC;
query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------+-------------------------+-------------------+---------------+----------+--------+----------
INSERT INTO test_table VALUES (100, 100);
|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
-(1 row)
+
+ SELECT coalesce(to_jsonb(array_agg(csa_from_one_node.*)), '[{}]'::JSONB)
+ FROM (
+ SELECT global_pid, worker_query AS is_worker_query, pg_stat_activity.* FROM
+ pg_stat_activity LEFT JOIN get_all_active_transactions() ON process_id = pid
+ ) AS csa_from_one_node;
+ |localhost | 57636|active | | |postgres|regression
+(2 rows)
step s3-view-worker:
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26692/workflows/3406e4b4-b686-4667-bec6-8253ee0809b1/jobs/765119
I intended to fix this with #6263, but the fix turned out to be
insufficient. This PR tries to address the issue by setting
distributedCommandOriginator correctly in more situations. However, even
with this change it's still possible to reproduce the flaky test in CI.
In any case this should fix at least some instances of this issue.
In passing this changes the isolation_citus_dist_activity test to allow
running it multiple times in a row.
PRE PG15, Renaming the parent triggers on partitioned tables doesn't
recurse to renaming the child triggers on the partitions as well.
In PG15, Renaming triggers on partitioned tables
recurses to renaming the triggers on the partitions as well.
Add an upgrade test to make sure we are not breaking anything
with distributed triggers on distributed partitioned tables.
Relevant PG commit:
80ba4bb383538a2ee846fece6a7b8da9518b6866
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.
We have lots of flaky tests in CI and most of these random failures are
very hard/impossible to reproduce locally. This adds a job definition to
CI that allows adding a temporary job to rerun the same test in CI a lot
of times. This will very often reproduce the random failures. If you
then try to change the test or code to fix the random failure, you can
confirm that it's indeed fixed by using this job.
A future improvement to this job would be to run it (or a variant of it)
automatically for every newly added test, and maybe even changed tests.
This is not implemented in this PR.
An example of this job running can be found here:
https://app.circleci.com/pipelines/github/citusdata/citus/26682/workflows/a2638385-35bc-443c-badc-7713a8101313
In commit 31faa88a4e I removed some features of the rebalance progress
monitor. I did this because the plan was to remove the foreground shard
rebalancer later in the PR that would add the background shard
rebalancer. So, I didn't want to spend time fixing something that we
would throw away anyway.
As it turns out we're not removing the foreground shard rebalancer after
all, so it made sens to fix the stuff that I broke. This PR does that.
For the most part this commit reverts the changes in commit 31faa88a4e.
It's not a full revert though, because it keeps the improved tests and
the changes to `citus_move_shard_placement`.
Before, this was the default mode for CustomScan providers.
Now, the default is to assume that they can't project.
This causes performance penalties due to adding unnecessary
Result nodes.
Hence we use the newly added flag, CUSTOMPATH_SUPPORT_PROJECTION
to get it back to how it was.
In PG15 support branch we created explain functions to ignore
the new Result nodes, so we undo that in this commit.
Relevant PG commit:
955b3e0f9269639fb916cee3dea37aee50b82df0
Sometimes in CI our multi_utilities test fails like this:
```diff
VACUUM (INDEX_CLEANUP ON, PARALLEL 1) local_vacuum_table;
SELECT CASE WHEN s BETWEEN 20000000 AND 25000000 THEN 22500000 ELSE s END size
FROM pg_total_relation_size('local_vacuum_table') s ;
size
----------
- 22500000
+ 39518208
(1 row)
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26641/workflows/5caea99c-9f58-4baa-839a-805aea714628/jobs/762870
Apparently VACUUM is not as reliable in cleaning up as we thought. This
increases the range of allowed values. Important to note is that the
range is still completely outside of the allowed range of the initial
size. So we know for sure that some data was cleaned up.
Sometimes in CI our adaptive_executor test would fail randomly with the
following error:
```diff
SELECT sum(result::bigint) FROM run_command_on_workers($$
SELECT count(*) FROM pg_stat_activity
WHERE pid <> pg_backend_pid() AND query LIKE '%8010090%'
$$);
sum
-----
- 4
+ 2
(1 row)
END;
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26665/workflows/40665680-0044-4852-8fe4-5fd628f9fb47/jobs/764371
This means that the low slow start interval did not have any effect on
the number of connections being opened. I could see two possibilities
for this to happen:
1. CI was slow and actually doing the start of the second connection. I
tried to solve this by doubling the time a query to the worker takes.
2. The second option is that the shards were queried in the oposite
order than we expect. This would mean that the first query to the
worker completes quickly because there's no, sleep because it doesn't
contain any rows. I tried to solve this option by adding a row to
each shard.
After trying to reproduce the random failure in CI it turned out that I
needed both of these fixes to resolve the random failure.
On CI our citus_split_shard_columnar_partitioned test would sometimes
randomly fail like this:
```diff
8970008 | colocated_dist_table | -2147483648 | 2147483647 | localhost | 57637
8970009 | colocated_partitioned_table | -2147483648 | 2147483647 | localhost | 57637
8970010 | colocated_partitioned_table_2020_01_01 | -2147483648 | 2147483647 | localhost | 57637
- 8970011 | reference_table | | | localhost | 57637
8970011 | reference_table | | | localhost | 57638
+ 8970011 | reference_table | | | localhost | 57637
(13 rows)
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26651/workflows/f695b4fb-ad81-46ff-b97e-0100e5d167ea/jobs/763517
This is a harmless diff due to a missing column in the order by list.
This fixes that by adding the nodeport as a tiebreaker.
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>
Sometimes in CI our isolation_citus_dist_activity test fails randomly
like this:
```diff
step s2-view-dist:
SELECT query, citus_nodename_for_nodeid(citus_nodeid_for_gpid(global_pid)), citus_nodeport_for_nodeid(citus_nodeid_for_gpid(global_pid)), state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE ALL(VALUES('%pg_prepared_xacts%'), ('%COMMIT%'), ('%BEGIN%'), ('%pg_catalog.pg_isolation_test_session_is_blocked%'), ('%citus_add_node%')) AND backend_type = 'client backend' ORDER BY query DESC;
query |citus_nodename_for_nodeid|citus_nodeport_for_nodeid|state |wait_event_type|wait_event|usename |datname
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+-------------------------+-------------------------+-------------------+---------------+----------+--------+----------
INSERT INTO test_table VALUES (100, 100);
|localhost | 57636|idle in transaction|Client |ClientRead|postgres|regression
-(1 row)
+
+ SELECT coalesce(to_jsonb(array_agg(csa_from_one_node.*)), '[{}]'::JSONB)
+ FROM (
+ SELECT global_pid, worker_query AS is_worker_query, pg_stat_activity.* FROM
+ pg_stat_activity LEFT JOIN get_all_active_transactions() ON process_id = pid
+ ) AS csa_from_one_node;
+ |localhost | 57636|active | | |postgres|regression
+(2 rows)
step s3-view-worker:
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26605/workflows/56d284d2-5bb3-4e64-a0ea-7b9b1626e7cd/jobs/760633
The reason for this is that citus_dist_stat_activity sometimes shows the
query that it uses itself to get the data from pg_stat_activity. This is
actually a bug, because it's a worker query and thus shouldn't show up
there. To try and solve this bug, we remove two small opportunities for a
race condition. These race conditions could happen when the backenddata
was marked as active, but the distributedCommandOriginator was not set
correctly yet/anymore. There was an opportunity for this to happen both
during connection start and shutdown.
Sometimes in CI our drop_partitioned_talbe test would fail with the
following error:
```diff
NOTICE: issuing SELECT worker_drop_distributed_table('drop_partitioned_table.child1')
NOTICE: issuing SELECT worker_drop_distributed_table('drop_partitioned_table.child1')
NOTICE: issuing DROP TABLE IF EXISTS drop_partitioned_table.child1_727001 CASCADE
-NOTICE: issuing SELECT pg_catalog.citus_internal_delete_colocation_metadata(100047)
-NOTICE: issuing SELECT pg_catalog.citus_internal_delete_colocation_metadata(100047)
+NOTICE: issuing SELECT pg_catalog.citus_internal_delete_colocation_metadata(100046)
+NOTICE: issuing SELECT pg_catalog.citus_internal_delete_colocation_metadata(100046)
ROLLBACK;
NOTICE: issuing ROLLBACK
NOTICE: issuing ROLLBACK
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26631/workflows/31536032-e1ba-493b-b12a-f40757f3a7d6/jobs/762170
For some reason the colocationid of the distributed partitioned table
would be one less than we expected. Why this happens I'm not sure, but
it seems fairly harmless that it does.
In an attempt to work around this flakyness I now reset the colocation
id sequence right before creating the table in question. This is good
practice in general, because it allows us to run the test successfully
using `check-minimal` and it also allows us to rerun it multiple times.
Our python based tests didn't always copy the normalized files after the
regress run. I had the problem where running the following command would
result in non-normalized files in the expected directory after running
our PG upgrade tests locally:
```
cp src/test/regress/{results,expected}/upgrade_list_citus_objects.out
```
This PR fixes that by always running `copy_modified` even if the tests
fail. The same was already being done for our perl based tests at the
end of the `pg_regress_multi.pl` file.
We currently do a `pg_relation_total_size('t1') + pg_relation_total_size('t2') + ..` on shard lists, especially when rebalancing the shards. This in some cases goes huge. With this PR, we basically use a SUM for all table sizes, instead of using thousands of pluses.
Sometimes in CI failure_online_move_shard_placement fails with the
following error:
```diff
SELECT citus.mitmproxy('conn.onQuery(query="^ALTER SUBSCRIPTION .* ENABLE").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
SELECT master_move_shard_placement(101, 'localhost', :worker_1_port, 'localhost', :worker_2_proxy_port);
-ERROR: canceling statement due to user request
+ERROR: tuple concurrently updated
+CONTEXT: while executing command on localhost:9060
-- failure on polling subscription state
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26441/workflows/dd6e3475-6121-47b3-aea3-4ac92be114f4/jobs/751476/steps
This error is not completely harmless, because based on the logs it mean
that our cleanup logic failed, which in turn means that replication
slots are left around:
```
2022-08-24 16:01:29.247 UTC [1219] ERROR: XX000: tuple concurrently updated
2022-08-24 16:01:29.247 UTC [1219] LOCATION: simple_heap_update, heapam.c:4179
2022-08-24 16:01:29.247 UTC [1219] STATEMENT: ALTER SUBSCRIPTION citus_shard_move_subscription_10 DISABLE
```
However, we have other mechanisms to clean up any leftovers in case of a
failed cleanup. So it's not that big of a problem.
The reason we run into this error is arguably because of a Postgres bug,
so I created a patch for Postgres that fixes this.
While we wait for this (or a similar) patch to be merged, this PR
disables the flaky test. There's still a test that tests in case of a
connection "kill" instead of a "cancel", so I don't think we lose very
important coverage by disabling this test. When trying to reproduce this
I only hit this issue in the cancel case, so I don't think there's a
need to disable the kill case for now.
In CI sometimes failure_connection_establishment would fail with the
following error:
```diff
-- cancel all connections to this node
SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')');
- mitmproxy
----------------------------------------------------------------------
-
-(1 row)
-
+ERROR: canceling statement due to user request
+CONTEXT: COPY mitmproxy_result, line 1: ""
+SQL statement "COPY mitmproxy_result FROM '/home/circleci/project/src/test/regress/tmp_check/mitmproxy.fifo'"
+PL/pgSQL function citus.mitmproxy(text) line 11 at EXECUTE
SELECT * FROM citus_check_cluster_node_health();
```
The reason for this is that the mitm command that was used is very
broad and doesn't actually do what the comment says. What happens is
that if any connection is made, the current backend is cancelled, which
is not the always the same as the backend that made the connection. My
assessment is that likely the maintenance daemon makes a connection to
the node while we are executing the mitmproxy command. The mitmproxy
command goes through, and then triggers a cancel of itself due to the
connection made by the maintenance daemon.
This PR simply removes this test, since it doesn't seem to test what it
intended to test anyway. There's also still the "kill" version of this
test, which does do the intended thing. So I don't think we lose
important coverage by removing this test.
Sometimes in CI multi_transaction_recovery would fail with the following
error:
```diff
SET LOCAL citus.defer_drop_after_shard_move TO OFF;
SELECT citus_move_shard_placement((SELECT * FROM selected_shard), 'localhost', :worker_1_port, 'localhost', :worker_2_port, shard_transfer_mode := 'block_writes');
- citus_move_shard_placement
----------------------------------------------------------------------
-
-(1 row)
-
+ERROR: could not find placement matching "localhost:57637"
+HINT: Confirm the placement still exists and try again.
COMMIT;
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26510/workflows/8269ea93-d9b4-4376-ae0e-8332a5c15fc6/jobs/755548
The reason for this was that when choosing `selected_shard` we didn't
ensure that it was actually located on the node that we were moving it
from. Instead we simply picked the first shard for the table that was
returned by the query.
To fix this issue this PR adds a filter to only choose shards that are
located on the intended node.
Our isolation_distributed_deadlock_detection test would fail randomly in
CI in three different ways.
The first type of failure looked like this:
```diff
check_distributed_deadlocks
---------------------------
t
(1 row)
-step s1-update-5: <... completed>
step s5-update-1: <... completed>
ERROR: canceling the transaction since it was involved in a distributed deadlock
+step s1-update-5: <... completed>
step s1-commit:
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26399/workflows/d213ee85-397a-467a-9ffb-39e4f44e6688/jobs/749533
This random change in output was harmless and happened because when the
deadlock detector cancelled a query, two queries would continue: The one
that was cancelled would throw an error (and thus complete), and the one
that was unblocked would now complete.
It was random which of the two the isolation tester would first detect
as completed. To resolve this PR starts using the ["marker" feature][1],
this allows us to make sure one of the steps won't be marked as
completed until the other one completed first.
The second random failure was very similar:
```diff
check_distributed_deadlocks
---------------------------
t
(1 row)
-step s2-update-2: <... completed>
-step s3-update-3: <... completed>
-ERROR: canceling the transaction since it was involved in a distributed deadlock
step s6-commit:
COMMIT;
step s5-update-6: <... completed>
+step s2-update-2: <... completed>
+step s3-update-3: <... completed>
+ERROR: canceling the transaction since it was involved in a distributed deadlock
step s5-commit:
```
Again a harmless difference in test output. In this case it's possible
that the deadlock detector would not detect the unblocked processes
right away, and would thus continue with to the next step. This step was
a commit on a session that was not blocked, and which thus could
complete without issues.
To solve this I changed the order of the commits at the end of the
permutation, to always have the first session that would commit be the
session that would be unblocked the last. This ensures that no commit
will ever be executed before completing all the queries.
The third issue was different and looked like this:
```diff
step s4-update-5: <... completed>
step s4-commit:
COMMIT;
+step s1-update-4: <... completed>
+isolationtester: canceling step s3-update-4 after 5 seconds
step s3-update-4: <... completed>
+ERROR: canceling statement due to user request
+step s2-update-2: <... completed>
step s3-commit:
COMMIT;
-step s2-update-2: <... completed>
-step s1-update-4: <... completed>
step s1-commit:
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26411/workflows/9089beec-4f0f-4027-b4ce-0e84889afc06/jobs/750143
The reason for this failure is not entirely clear to me, but I was able
to remove the flakyness without impacting the goal of the test. What was
happening was that both `s1` and `s3` were waiting for `s4` to commit
and release it's lock on the row 4. For some reason it wasn't
deterministic which of the two sessions would be granted the lock after
it was released by row 4. The test expected `s3` to be granted the lock,
but sometimes it would be granted to `s1` instead. Which would in turn
cause `s3` to still be blocked.
To solve this I simply removed `s1` completely from this test. It wasn't
actually part of the cycle that the deadlock detector should detect and
was an unrelated appendage:
```mermaid
graph TD;
s2-->s3;
s3-->s4;
s1-->s4;
s4-->s5;
s5-->s6;
s6-->s5;
```
By removing `s1` completely there was no contention for the lock and
`s3` could always acquire it.
[1]: a73d6c87f2/src/test/isolation/README (L163-L188)
In CI multi_utilities would sometimes fail randomly with this error:
```diff
VACUUM (INDEX_CLEANUP ON, PARALLEL 1) local_vacuum_table;
SELECT pg_size_pretty( pg_total_relation_size('local_vacuum_table') );
pg_size_pretty
----------------
- 21 MB
+ 22 MB
(1 row)
```
Source: https://app.circleci.com/pipelines/github/citusdata/citus/26459/workflows/da47d9b6-f70b-49fe-806f-5ebf75bf0b11/jobs/752482
This is a harmless change in output where the relation size after
vacuuming was slightly more than we expected. This changes the size
checks for the local_vacuum_table to allow a wider range of values.
It uses the same trick as #6216 to show the actual value when it's
outside this valid range, which is useful if this test ever starts
failing again.
When trying to fix#6245 I realized that multi_utilities was leaking
some tables that it created during the test. This fixes that by
creating all these tables in a schema that's dedicated for this test.
When running `make check-base` locally it would fail with two different
errors.
The first one was this:
```diff
SELECT create_distributed_table('pg_class', 'relname');
-ERROR: cannot create a citus table from a catalog table
+ERROR: deadlock detected
+DETAIL: Process 28950 waits for ExclusiveLock on relation 16551 of database 16384; blocked by process 28951.
+Process 28951 waits for RowExclusiveLock on relation 1259 of database 16384; blocked by process 28950.
+HINT: See server log for query details.
SELECT create_reference_table('pg_class');
```
This happened because multi_behavioral_analytics_create_table and
multi_create_table were being run in parallel. Running them separately
resolved this issue.
The second one was this:
```diff
CREATE OR REPLACE FUNCTION wait_until_metadata_sync(timeout INTEGER DEFAULT 15000)
RETURNS void
LANGUAGE C STRICT
AS 'citus';
+ERROR: duplicate key value violates unique constraint "pg_proc_proname_args_nsp_index"
+DETAIL: Key (proname, proargtypes, pronamespace)=(wait_until_metadata_sync, 23, 2200) already exists.
-- Add some helper functions for sending commands to mitmproxy
```
Which was because failure_test_helpers and multi_test_helpers were
trying to create the same function at the exact same time. The easy fix
here is to simply not create this function in the failure_test_helpers
file. This is fine, because any test schedule that runs
failure_test_helpers also runs multi_test_helpers.
I upgraded my OS to Ubuntu 22.04 a while back and since then some tests
order output slightly differently. I think it might be because of the
glibc upgrade that changed ordering for things like underscores and
spaces.
Changing the locale to C.UTF-8 solves this issue.