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.
* Alter_distributed_table colocateWith:none bug fix for partitioned tables.
* Regression tests added for alter_distributed_table colocateWith:none for partitioned tables
* Update query comparision to be more accurate
Postgres supports JSON_TABLE feature on PG 15.
We treat JSON_TABLE the same as correlated functions (e.g., recurring tuples).
In the end, for multi-shard JSON_TABLE commands, we apply the same
restrictions as reference tables (e.g., cannot be in the outer part of
an outer join etc.)
Co-authored-by: Onder Kalaci <onderkalaci@gmail.com>
* Adjust configure script to allow PG15
* Adds copy of ruleutils_14.c as ruleutils_15.c
* Uses get_namespace_name_or_temp in ruleutils_15.c
Relevant PG commit:
48c5c9068211e0a04fd9553c8714b2821ed3ad17
* Clean up code using "(expr) ? true : false" in ruleutils_15.c
Relevant PG commit:
fd0625c7a9c679c0c1e896014b8f49a489c3a245
* Change varno from Index (unsigned int) to int in ruleutils_15.c
Relevant PG commit:
e3ec3c00d85bd2844ffddee83df2bd67c4f8297f
* Adds find_recursive_union to ruleutils_15.c
Relevant PG commit:
3f50b82639637c9908afa2087de7588450aa866b
* Fix display of SQL-std func's args in INSERT/SELECT in ruleutils_15.c
Relevant PG commit:
a8d8445a7b2f80f6d0bfe97b19f90bd2cbef8759
* Fix ruleutils_15.c's dumping of whole-row Vars in more contexts
Relevant PG commit:
43c2175121c829c8591fc5117b725f1f22bfb670
* Fix assorted missing logic for GroupingFunc nodes in ruleutils_15.c
Relevant PG commit:
2591ee8ec44d8cbc8e1226550337a64c684746e4
* Adds grammar support for SQL/JSON clauses in ruleutils_15.c
Relevant PG commit:
f79b803dcc98d707450e158db3638dc67ff8380b
* Adds SQL/JSON constructors to ruleutils_15.c
Relevant PG commits:
f4fb45d15c59d7add2e1b81a9d477d0119a9691a
cc7401d5ca498a84d9b47fd2e01cebd8e830e558
* Adds support for MERGE in ruleutils_15.c
Relevant PG commit:
7103ebb7aae8ab8076b7e85f335ceb8fe799097c
* Add IS JSON predicate to ruleutils_15.c
Relevant PG commit:
33a377608fc29cdd1f6b63be561eab0aee5c81f0
* Add SQL/JSON query functions to ruleutils_15.c
Relevant PG commit:
1a36bc9dba8eae90963a586d37b6457b32b2fed4
* Adds three different SQL/JSON values to ruleutils_15.c
Relevant PG commits:
606948b058dc16bce494270eea577011a602810e
49082c2cc3d8167cca70cfe697afb064710828ca
* Adds JSON table functions in ruleutils_15.c
Relevant PG commit:
4e34747c88a03ede6e9d731727815e37273d4bc9
* Add PLAN function for JSON table in ruleutils_15.c
Relevant PG commit:
fadb48b00e02ccfd152baa80942de30205ab3c4f
* Remove extra blank lines before block-closing braces ruleutils_15.c
Relevant PG commit:
24d2b2680a8d0e01b30ce8a41c4eb3b47aca5031
* set_deparse_plan: Reuse variable to appease Coverity ruleutils_15.c
Relevant PG commit:
e70813fbc4aaca35ec012d5a426706bd54e4acab
* Mechanical code beautification ruleutils_15.c
Relevant PG commit:
23e7b38bfe396f919fdb66057174d29e17086418
* Rename value_type to item_type in ruleutils_15.c
Relevant PG commit:
3ab9a63cb638a1fd99475668e2da9c237495aeda
* Show 'AS "?column?"' explicitly when it's important in ruleutils_15.c
Relevant PG commit:
c7461fc25558832dd347a9c8150b0f1ed85e36e8
* Fix ruleutils_15.c issues with dropped cols in funcs-returning-composite
Relevant PG commit:
c1d1e8469c77ce6b8e5310955580b4a3eee7fe96
* Change comment regarding functions returning composite in ruleutils_15.c
Relevant PG commit:
c2fa113ddb1117b1f03e91960f65d5d7d8a90270
* Replace int nodes with bool nodes where needed
In PG15, Boolean nodes are added. Pre PG15, internal Boolean values
in Create Role commands were represented by Integer nodes. This
commit replaces int nodes logic with bool nodes logic where needed.
Mostly there are CREATE ROLE logic changes.
Relevant PG commit:
941460fcf731a32e6a90691508d5cfa3d1f8eeaf
* Handle new option colliculocale in CREATE COLLATION logic
In PG15, there is an added option to use ICU as global locale provider.
pg_collation has three locale-related fields: collcollate and collctype,
which are libc-related fields, and a new one colliculocale, which is the
ICU-related field. Only the libc-related fields or the ICU-related field
is set, never both.
Relevant PG commits:
f2553d43060edb210b36c63187d52a632448e1d2
54637508f87bd5f07fb9406bac6b08240283be3b
* Add PG15 tests to CI using test images that have 15beta2 (#6093)
* Change warning message in pg_signal_backend()
Relevant PG commit:
7fa945b857cc1b2964799411f1633468826861ff
* Revert "Add missing ifdef for PG 15"
This reverts commit c7b51025ab.
* Fixes tests for ALTER TRIGGER RENAME consistency for part. tables
Relevant PG commit:
80ba4bb383538a2ee846fece6a7b8da9518b6866
* Prevent creating child triggers on partitions when adding new node
Pre PG15, tgisinternal is true for a "child" trigger on a partition
cloned from the trigger on the parent.
In PG15, tgisinternal is false in that case. However, we don't want to
create this trigger on the partition since it will create a conflict
when we try to attach the partition to the parent table:
ERROR: trigger "..." for relation "{partition_name}" already exists
Relevant PG commit:
f4566345cf40b068368cb5617e61318da60676ec
* Fix tests for generated columns dependency changes
In PG15, For GENERATED columns, all dependencies of the generation
expression are recorded as NORMAL dependencies of the column itself.
This requires CASCADE to drop generated cols with the original col.
PRE PG15, dependencies were recorded as AUTO, with which
generated columns are silently dropped with the original column.
Relevant PG commit:
cb02fcb4c95bae08adaca1202c2081cfc81a28b5
* Explicitly cast catalog "char" column to text before concatenation
Relevant PG commit:
07eee5a0dc642d26f44d65c4e6263304208e8583
* Remove 'AS "?column?"' from test outputs
There were some instances in the following tst outputs
in planning debug outputs where AS "?column?" is added.
We add a normalization rule to remove it as it is not
important.
cte_inline.out
recursive_relation_planning_restriction_pushdown.out
Relevant PG commit:
c7461fc25558832dd347a9c8150b0f1ed85e36e8
* Use pg_backup_stop(PG15) instead of pg_stop_backup(PG<15)
Add an alternative test output because of the change in the
backup modes of Postgres. Specifically here, there is a renaming
issue: pg_stop_backup PRE PG15 vs pg_backup_stop PG15+
The alternative output can be deleted when we drop support for PG14
Relevant PG commit:
39969e2a1e4d7f5a37f3ef37d53bbfe171e7d77a
* Adds citus.mitmfifo GUC
Previously we setting this configuration parameter
in the fly for failure tests schedule.
However, PG15 doesn't allow that anymore: reserved prefixes
like "citus" cannot be used to set non-existing GUCs.
Relevant PG commit:
88103567cb8fa5be46dc9fac3e3b8774951a2be7
* Handles EXPLAIN output diffs in PG15 - Extra result lines
To handle extra "Result" lines in explain outputs, we add explain
method to multi_test_helpers.sql file
- plan_without_result_lines() is added for cases where we want the
whole explain output with only "Result" lines removed
* Handles EXPLAIN output diffs in PG15, Hash Agg/Join leverage
To handle differences in usage of GroupAggregate vs HashAggregate
or Merge Join vs Hash join in cases where this detail doesn't
seem to matter, we use coordinator_plan().
- coordinator_plan() is updated to remove "Result" lines
There are some cases where we have subplans so we add a new
function that prints all Task Count lines as well
- coordinator_plan_with_subplans()
Still not sure of the relevant PG commit
Could be db0d67db2401eb6238ccc04c6407a4fd4f985832
but disabling enable_group_by_reordering didn't help.
* Handles EXPLAIN output diffs in PG15: enable_group_by_reordering
Relevant PG commit
db0d67db2401eb6238ccc04c6407a4fd4f985832
* Normalizes Memory Usage, Buckets, Batches for PG15 explain diffs
We create a new function in multi_test_helpers, which is similar
to explain_merge function in PG15. This explain helper function
normalies Memory Usage, Buckets and Batches, and we use it in the
tests which give a different output for PG15.
* Bump test images to 15beta3 (#6172)
* Omit namespace in post-copy errmsg
Relevant PG commit:
069d33d0c5a021601245e44df77a0423ddd69359
* Handles EXPLAIN output diffs in PG15: extra arrows&result lines
To handle extra "->" arrows resulting from extra Result lines
in explain outputs, we add the following explain method to
multi_test_helpers.sql file
- plan_without_arrows() is added for cases where we want the
whole explain output without arrows and without Result lines
* Alters public schema's owner to pg_database_owner in PG15
In PG15, public schema is owned by pg_database_owner role.
In multi_extension, we drop and recreate the ppublic schema,
hence its owner become the default user in our tests, postgres.
Change that to pg_database_owner for PG15 consistency.
This results in alternative test output for public schema grants
in the following test:
grant_on_schema_propagation.sql
Relevant PG commit: b073c3ccd06e4cb845e121387a43faa8c68a7b62
* Add alternative test outputs for change in Insert Select display
citus_local_tables_queries.sql
coordinator_shouldhaveshards.sql
cte_inline.sql
insert_select_repartition.sql
intermediate_result_pruning.sql
local_shard_execution.sql
local_shard_execution_replicated.sql
multi_deparse_shard_query.sql
multi_insert_select.sql
multi_insert_select_conflict.sql
multi_mx_insert_select_repartition.sql
mx_coordinator_shouldhaveshards.sql
single_node.sql
Relevant PG commit:
a8d8445a7b2f80f6d0bfe97b19f90bd2cbef8759
* Fixes columnar tap tests for PG15
In PG15, Perl test modules have been moved to a new namespace.
Also, postgres node new() and get_new_node() methods have been
unified to one method: new()
We create separate tap tests for PG13/14 and PG15+
and update the Makefiles accordingly.
Relevant PG commits:
201a76183e2056c2217129e12d68c25ec9c559c8
b3b4d8e68ae83f432f43f035c7eb481ef93e1583
* Handles EXPLAIN output diffs in PG15: HashAgg Leverage,alt. output
Still not sure of the relevant PG commit
Could be db0d67db2401eb6238ccc04c6407a4fd4f985832
but disabling enable_group_by_reordering didn't help.