Commit Graph

357 Commits (3f67efb2c143f5784ab7050edcd9c199fd425ca3)

Author SHA1 Message Date
Naisila Puka bd91df298f
Fixes ConnectionModifiedPlacement output for a failed transaction (#5198) 2021-08-31 18:58:46 +03:00
Nils Dijk dfc950ce1e
Fix a segfault caused by use after free in ConnectionsPlacementHash (#5170)
DESCRIPTION: Fix a segfault caused by use after free in ConnectionsPlacementHash

Fix a segfault caused by retaining data in any of the hashmaps making up the Placement Connection Management.

We have seen production systems segfault due to random data referenced from ConnectionPlacementHash.
On investigation we found that the backends segfaulting on this had OOM errors closely prior to the segfault.
It has shown there are at least 15 places where an allocation can OOM that would cause ConnectionPlacementHash to retain pointers to memory from contexts that are subsequently freed. This would reproduce the segfault we have observed in production.

Conditions for these allocations are:
 - allocated after first call to `AssociatePlacementWithShard`: https://github.com/citusdata/citus/blob/v10.0.3/src/backend/distributed/connection/placement_connection.c#L880-L881
 - allocated before `StartNodeUserDatabaseConnection`: https://github.com/citusdata/citus/blob/v10.0.3/src/backend/distributed/connection/connection_management.c#L291

At least 15 points of memory allocation (which could fail) are between the callsites of both in a primary key lookup on a reference table - where we have seen an OOM cause a segfault moments later.

Instead of leaving any references in ConnectionPlacementHash, ConnectionShardHash and ColocatedPlacementsHash that could retain any pointers that are freed due to the TopTransactionContext being reset we clear all these hashes irregardless of the state of CurrentCoordinatedTransactionState.

Downside is that on any transaction abort we will now iterate through 4 hashmaps and clear their contents. Given that they are either already empty, which should cause a quick iteration, or non-empty, causing segfaults in subsequent executions, this overhead seems reasonable.

A better solution would be to move the creation of these hashmaps so they would live in the TopTransactionContext themself, assuming their contents would never outlive a transaction. This needs more investigation and is an involved refactor Hence fixing this quickly here.
2021-08-17 17:42:35 +02: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
Onur Tirtir 3d11c0f9ef Merge remote-tracking branch 'origin/master' into columnar-index
Conflicts:
	src/test/regress/expected/columnar_empty.out
	src/test/regress/expected/multi_extension.out
2021-06-16 20:23:50 +03:00
Jelte Fennema b1cad26ebc Move CheckCitusVersion to the top of each function
Previously this was usually done after argument parsing. This can cause
SEGFAULTs if the number or type of arguments changes in a new version.
By checking that Citus version is correct before doing any argument
parsing we protect against these types of issues. Issues like this have
occurred in pg_auto_failover, so it's not just a theoretical issue.

The main reason why these calls were not at the top of functions is
really just historical. It was because in the past we didn't allow
statements before declarations. Thus having this check before the
argument parsing would have only been possible if we first declared all
variables.

In addition to moving existing CheckCitusVersion calls it also adds
these calls to rebalancer related functions (they were missing there).
2021-06-01 17:43:46 +02:00
Ahmet Gedemenli 69d39c0e8b Fix relname null bug when parallel execution 2021-06-01 14:14:35 +03:00
SaitTalhaNisanci a4944a2102
Rename CoordinatedTransactionShouldUse2PC (#4995) 2021-05-21 18:57:42 +03:00
jeff-davis 7b9aecff21 Columnnar: metapage changes. (#4907)
* Columnar: introduce columnar storage API.

This new API is responsible for the low-level storage details of
columnar; translating large reads and writes into individual block
reads and writes that respect the page headers and emit WAL. It's also
responsible for the columnar metapage, resource reservations (stripe
IDs, row numbers, and data), and truncation.

This new API is not used yet, but will be used in subsequent
forthcoming commits.

* Columnar: add columnar_storage_info() for debugging purposes.

* Columnar: expose ColumnarMetadataNewStorageId().

* Columnar: always initialize metapage at creation time.

This avoids the complexity of dealing with tables where the metapage
has not yet been initialized.

* Columnar: columnar storage upgrade/downgrade UDFs.

Necessary upgrade/downgrade step so that new code doesn't see an old
metapage.

* Columnar: improve metadata.c comment.

* Columnar: make ColumnarMetapage internal to the storage API.

Callers should not have or need direct access to the metapage.

* Columnar: perform resource reservation using storage API.

* Columnar: implement truncate using storage API.

* Columnar: implement read/write paths with storage API.

* Columnar: add storage tests.

* Revert "Columnar: don't include stripe reservation locks in lock graph."

This reverts commit c3dcd6b9f8.

No longer needed because the columnar storage API takes care of
concurrency for resource reservation.

* Columnar: remove unnecessary lock when reserving.

No longer necessary because the columnar storage API takes care of
concurrent resource reservation.

* Add simple upgrade tests for storage/ branch

* fix multi_extension.out

Co-authored-by: Onur Tirtir <onurcantirtir@gmail.com>
2021-05-10 20:16:46 +03: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
SaitTalhaNisanci 03832f353c Drop postgres 11 support 2021-03-25 09:20:28 +03:00
Marco Slot 6c5d263b7a Remove unnecessary AtEOXact_Files call 2021-03-15 09:34:02 +01:00
Onder Kalaci e65e72130d Rename use -> shouldUse
Because setting the flag doesn't necessarily mean that we'll
use 2PC. If connections are read-only, we will not use 2PC.
In other words, we'll use 2PC only for connections that modified
any placements.
2021-03-12 08:29:43 +00:00
Onder Kalaci 6a7ed7b309 Do not trigger 2PC for reads on local execution
Before this commit, Citus used 2PC no matter what kind of
local query execution happens.

For example, if the coordinator has shards (and the workers as well),
even a simple SELECT query could start 2PC:
```SQL

WITH cte_1 AS (SELECT * FROM test LIMIT 10) SELECT count(*) FROM cte_1;
```

In this query, the local execution of the shards (and also intermediate
result reads) triggers the 2PC.

To prevent that, Citus now distinguishes local reads and local writes.
And, Citus switches to 2PC only if a modification happens. This may
still lead to unnecessary 2PCs when there is a local modification
and remote SELECTs only. Though, we handle that separately
via #4587.
2021-03-12 08:29:43 +00:00
Naisila Puka 196064836c
Skip 2PC for readonly connections in a transaction (#4587)
* Skip 2PC for readonly connections in a transaction

* Use ConnectionModifiedPlacement() function

* Remove the second check of ConnectionModifiedPlacement()

* Add order by to prevent flaky output

* Test using pg_dist_transaction
2021-03-10 20:01:37 +03:00
Philip Dubé 4e22f02997 Fix various typos due to zealous repetition 2021-03-04 19:28:15 +00:00
Hadi Moshayedi c3dcd6b9f8 Columnar: don't include stripe reservation locks in lock graph. 2021-02-10 10:20:20 -08:00
Onur Tirtir cacb76d2c6
Not mention citus local tables in error messages (#4579) 2021-01-27 12:36:53 +03:00
Hadi Moshayedi bc01c795a2 Reland #4419 2021-01-19 07:48:47 -08:00
Marco Slot d9f175532b Revert "Trigger metadata sync at transaction commit"
This reverts commit a2c73bef27.
2021-01-07 10:30:00 +01:00
Onder Kalaci 2fe158961b Remove "WarnAboutLeakedPreparedTransaction" function
We used to need WarnAboutLeakedPreparedTransaction()
as we didn't have auto 2PC recovery. But, we long have
2PC recovery by https://github.com/citusdata/citus/pull/1574

So, we don't need anymore.
2021-01-06 15:48:58 +03:00
Hadi Moshayedi a2c73bef27 Trigger metadata sync at transaction commit 2020-12-24 08:28:38 -08:00
Ahmet Gedemenli 7577821920 Fix transaction name length calculation 2020-12-07 12:34:15 +03:00
Ahmet Gedemenli 936775e8e3 Delete transactions when removing node
With this commit, we delete entries in pg_dist_transaction
for the primary nodes that are removed by `master_remove_node`.
2020-12-07 11:35:20 +03:00
Onder Kalaci 629ecc3dee Add the infrastructure to count the number of client backends
Considering the adaptive connection management
improvements that we plan to roll soon, it makes it
very helpful to know the number of active client
backends.

We are doing this addition to simplify yhe adaptive connection
management for single node Citus. In single node Citus, both the
client backends and Citus parallel queries would compete to get
slots on Postgres' `max_connections` on the same Citus database.

With adaptive connection management, we have the counters for
Citus parallel queries. That helps us to adaptively decide
on the remote executions pool size (e.g., throttle connections
if necessary).

However, we do not have any counters for the total number of
client backends on the database. For single node Citus, we
should consider all the client backends, not only the remote
connections that Citus does.

Of course Postgres internally knows how many client
backends are active. However, to get that number Postgres
iterates over all the backends. For examaple, see [pg_stat_get_db_numbackends](8e90ec5580/src/backend/utils/adt/pgstatfuncs.c (L1240))
where Postgres iterates over all the backends.

For our purpuses, we need this information on every connection
establishment. That's why we cannot affort to do this kind of
iterattion.
2020-11-25 19:19:24 +01:00
Onur Tirtir f80f4839ad Remove unused functions that cppcheck found 2020-10-19 13:50:52 +03:00
SaitTalhaNisanci e7cd1ed0ee
Not take ShareUpdateExlusiveLock on pg_dist_transaction (#4184)
* Not take ShareUpdateExlusiveLock on pg_dist_transaction

We were taking ShareUpdateExlusiveLock on pg_dist_transaction during
recovery to prevent multiple recoveries happening concurrenly. VACUUM(
not FULL) also takes ShareUpdateExclusiveLock, and they can conflict. It
seems that VACUUM will skip the table if there is a conflicting lock
already taken unless it is doing the vacuum to prevent id wraparound, in
which case there can be a deadlock. I guess the deadlock happens if:

- VACUUM takes a lock on pg_dist_transaction and is done for id
wraparound problem
- The transaction in the maintenance tries to take a lock but
cannot as that conflicts with the lock acquired by VACUUM
- The transaction in the maintenance daemon has a very old xid hence
VACUUM cannot proceed.

If we take a row exclusive lock in transaction recovery then it wouldn't
conflict with VACUUM hence it could proceed so the deadlock would be
resolved. To prevent concurrent transaction recoveries happening, an
advisory lock is taken with ShareUpdateExlusiveLock as before.

* Use CITUS_OPERATIONS tag
2020-09-21 15:20:38 +03:00
Onur Tirtir 3a73fba810 Apply planner changes for citus local tables 2020-09-09 11:51:18 +03:00
Onur Tirtir 0b1cc118a9 Adapt other cache entry changes for citus local tables 2020-09-09 11:50:55 +03:00
Onur Tirtir ba208eae4d
Record non-distributed table accesses in local executor (#4139) 2020-09-07 18:19:08 +03:00
SaitTalhaNisanci 366461ccdb
Introduce cache entry/table utilities (#4132)
Introduce table entry utility functions

Citus table cache entry utilities are introduced so that we can easily
extend existing functionality with minimum changes, specifically changes
to these functions. For example IsNonDistributedTableCacheEntry can be
extended for citus local tables without the need to scan the whole
codebase and update each relevant part.

* Introduce utility functions to find the type of tables

A table type can be a reference table, a hash/range/append distributed
table. Utility methods are created so that we don't have to worry about
how a table is considered as a reference table etc. This also makes it
easy to extend the table types.

* Add IsCitusTableType utilities

* Rename IsCacheEntryCitusTableType -> IsCitusTableTypeCacheEntry

* Change citus table types in some checks
2020-09-02 22:26:05 +03:00
Sait Talha Nisanci b641f63bfd Use CMDTAG_SELECT_COMPAT
CMDTAG_SELECT exists in PG12 hence defining a MACRO such as
CMDTAG_SELECT -> "SELECT" is not possible. I chose CMDTAG_SELECT_COMPAT
because with the COMPAT suffix it is explicit that it maps to different
things in different versions and also has a less chance of mapping
something irrevelant. For example if we used SELECT as a macro, then it
would map every SELECT to whatever it is mapping to, which might have
unexpected/undesired behaviour.
2020-08-04 15:38:13 +03:00
Sait Talha Nisanci 1112b254a7 adapt recently added code for pg13
This commit mostly adds pg_get_triggerdef_command to our ruleutils_13.
This doesn't add anything extra for ruleutils 13 so it is basically a copy
of the change on ruleutils_12
2020-08-04 15:18:27 +03:00
Sait Talha Nisanci 01632c56a0 Change utils/hashutils.h to common/hashfn.h for PG >= 13
Commit on postgres side:
05d8449e73694585b59f8b03aaa087f04cc4679a

Command on postgres side:
git log --all --grep="hashutils"

include common/hashfn.h for pg >= 13

tag_hash was moved from hsearch.h to hashutils.h then to hashfn.h

Commits on Postgres side:
9341c783cc42ffae5860c86bdc713bd47d734ffd
2020-08-04 15:10:22 +03:00
Sait Talha Nisanci bf831d2e59 Use table_openXXX methods in the codebase
With PG13 heap_* (heap_open, heap_close etc) are replaced with table_*
(table_open, table_close etc).

It is better to use the new table access methods in the codebase and
define the macros for the previous versions as we can easily remove the
macro without having to change the codebase when we drop the support for
the old version.

Commits that introduced this change on Postgres:
f25968c49697db673f6cd2a07b3f7626779f1827
e0c4ec07284db817e1f8d9adfb3fffc952252db0
4b21acf522d751ba5b6679df391d5121b6c4a35f

Command to see relevant commits on Postgres side:
git log --all --grep="heap_open"
2020-08-04 15:10:22 +03:00
Onder Kalaci eeb8c81de2 Implement shared connection count reservation & enable `citus.max_shared_pool_size` for COPY
With this patch, we introduce `locally_reserved_shared_connections.c/h` files
which are responsible for reserving some space in shared memory counters
upfront.

We sometimes need to reserve connections, but not necessarily
establish them. For example:
-  COPY command should reserve connections as it cannot know which
   connections it needs in which order. COPY establishes connections
   as any input data hits the workers. For example, for router COPY
   command, it only establishes 1 connection.

   As discussed here (https://github.com/citusdata/citus/pull/3849#pullrequestreview-431792473),
   COPY needs to reserve connections up-front, otherwise we can end
   up with resource starvation/un-detected deadlocks.
2020-08-03 18:51:40 +02:00
Marco Slot 5fbb925df1 Remove level asserts in abort handler 2020-07-12 22:54:35 +02: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
Hadi Moshayedi 3651fc64ee Fix Subtransaction memory leak 2020-07-09 12:33:39 -07:00
SaitTalhaNisanci 96adce77d6
rename node/worker utilities (#4003)
The names were not explicit about what they do, and we have many
misusages in the codebase, so they are renamed to be more explicit.
2020-07-09 15:30:35 +03:00
Jelte Fennema 392c5e2c34
Fix wrong cancellation message about distributed deadlocks (#3956) 2020-06-30 14:57:46 +02:00
Jelte Fennema 02fa942be1
Fix assertion error when rolling back to savepoint (#3868)
It was possible to get an assertion error, if a DML command was
cancelled that opened a connection and then "ROLLBACK TO SAVEPOINT" was
used to continue the transaction. The reason for this was that canceling
the transaction might leave the `claimedExclusively` flag on for (some
of) it's connections.

This caused an assertion failure because `CanUseExistingConnection`
would return false and a new connection would be opened, and then there
would be two connections doing DML for the same placement. Which is
disallowed. That this situation caused an assertion failure instead of
an error, means that without asserts this could possibly result in some
visibility bugs, similar to the ones described
https://github.com/citusdata/citus/issues/3867
2020-06-30 11:31:46 +02:00
Marco Slot d1bab78d79 Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
Jelte Fennema 0e12d045b1
Support use of binary protocol in between nodes (#3877)
This can save a lot of data to be sent in some cases, thus improving
performance for which inter query bandwidth is the bottleneck.
There's some issues with enabling this as default, so that's currently not done.
2020-06-12 15:02:51 +02:00
Jelte Fennema f4791fcb10
Remove SwallowErrors by using PathNameDeleteTemporaryDir (#3893)
This is a different version of #3634. It also removes SwallowErrors, but
instead of modifying our own functions to not throw errors, it uses the
postgres built in `PathNameDeleteTemporaryDir` function. This function
does not throw errors.

Since this change is for a bugfix, I tried to minimize the changes.

PRs with the following changes would be good to do separately from this
PR:
1. Use PathName(Create|Open|Delete)Temporary(File|Dir) to open and
   remove all files/dirs instead of our own custom file functions.
2. Prefix our outmost files/directories with `PG_TEMP_FILE_PREFIX` so
   that they are identified by Postgres as temporary files, which will be
   removed at postmaster start. This way we do not have to do this cleanup
   ourselves.
3. Store the files in the temporary table space if it exists.

Fixes #3634
Fixes #3618
2020-06-10 17:04:07 +02:00
Hadi Moshayedi 5cdfa9f571 Implement EXPLAIN ANALYZE udfs.
Implements worker_save_query_explain_analyze and worker_last_saved_explain_analyze.

worker_save_query_explain_analyze executes and returns results of query while
saving its EXPLAIN ANALYZE to be fetched later.

worker_last_saved_explain_analyze returns the saved EXPLAIN ANALYZE result.
2020-06-09 10:02:05 -07:00
Philip Dubé c0515dcd67 This prepares for routing modifying CTEs, where modLevel should not be used to infer whether a plan is a select or not
SELECT_TASK is renamed to READ_TASK as a SELECT with modifying CTEs will be a MODIFYING_TASK

RouterInsertJob: Assert originalQuery->commandType == CMD_INSERT
CreateModifyPlan: Assert originalQuery->commandType != CMD_SELECT

Remove unused function IsModifyDistributedPlan

DistributedExecution, ExecutionParams, DistributedPlan: Rename hasReturning to expectResults
SELECTs set expectResults to true

Rename CreateSingleTaskRouterPlan to CreateSingleTaskRouterSelectPlan
2020-05-20 17:26:12 +00:00
Jelte Fennema c6f5d5fe88
Add some asserts to pass static analysis (#3805) 2020-04-29 11:19:11 +02:00
Hadi Moshayedi 59b9a4e5a1 Detect deadlocks in replicate_reference_tables() 2020-04-15 11:06:18 -07:00
Marco Slot 8b83306a27 Issue worker messages with the same log level 2020-04-14 21:08:25 +02:00
Onder Kalaci aa6b641828 Throttle connections to the worker nodes
With this commit, we're introducing a new infrastructure to throttle
connections to the worker nodes. This infrastructure is useful for
multi-shard queries, router queries are have not been affected by this.

The goal is to prevent establishing more than citus.max_shared_pool_size
number of connections per worker node in total, across sessions.

To do that, we've introduced a new connection flag OPTIONAL_CONNECTION.
The idea is that some connections are optional such as the second
(and further connections) for the adaptive executor. A single connection
is enough to finish the distributed execution, the others are useful to
execute the query faster. Thus, they can be consider as optional connections.
When an optional connection is not allowed to the adaptive executor, it
simply skips it and continues the execution with the already established
connections. However, it'll keep retrying to establish optional
connections, in case some slots are open again.
2020-04-14 10:27:48 +02:00
SaitTalhaNisanci 3dc7cad754
use an enum for local execution status (#3733)
We have two variables that are related to local execution status.
TransactionAccessedLocalPlacement and
TransactionConnectedToLocalGroup. Only one of these fields should be
set, however we didn't have any check for this contraint and it was
error prone.

What those two variables are used is that we are trying to understand if
we should use local execution, the current session, or if we should be
using a connection to execute the current query, therefore the tasks. In
the enum, now it is more clear what these variables mean.

Also, now we have a method to change the local execution status. The
method will error if we are trying to transition from a state to a wrong
state. This will help us avoid problems.
2020-04-09 19:11:04 +03:00
SaitTalhaNisanci fa88046ce1
test that we don't leak intermediate schemas (#3737)
* test that we don't leak intermediate schemas

We have tests to make sure that we don't intermediate any intermediate
files, tables etc but we don't test if we are leaking schemas. It makes
sense to test this as well.

* remove all repartition schemas in case of error

This solution is not an ideal one but it seems to be doing the job.
We should have a more generic solution for the cleanup but it seems that
putting the cleanup in the abort handler is dangerous and it was
crashing.
2020-04-09 12:17:41 +03:00
Marco Slot 924cd7343a Defer reference table replication to shard creation time 2020-04-08 12:41:36 -07:00
SaitTalhaNisanci ba01f3457a
use macros for pg versions instead of hardcoded values (#3694)
3 Macros are defined for removing the hardcoded pg versions.
PG_VERSION_11, PG_VERSION_12 and PG_VERSION_13.
2020-04-01 17:01:52 +03:00
Onur Tirtir 8ebb8ef31d use PG_USED_FOR_ASSERTS_ONLY 2020-03-25 11:01:33 +03:00
Philip Dubé 7cdfa1daab Rename LookupCitusTableCacheEntry to GetCitusTableCacheEntry, LookupLookupCitusTableCacheEntry back to LookupCitusTableCacheEntry 2020-03-08 14:08:23 +00:00
Philip Dubé a7cca1bcde Rename DistTableCacheEntry to CitusTableCacheEntry 2020-03-07 14:08:03 +00:00
Philip Dubé bec58000d6 Given IsDistributedTableRTE, there's ambiguity in what DistributedTable means
Elsewhere we used DistributedTable to include reference tables
Marco suggested we use CitusTable for distributed & reference tables

So renaming:
- IsDistributedTable -> IsCitusTable
- IsDistributedTableViaCatalog -> IsCitusTableViaCatalog
- DistributedTableCacheEntry -> CitusTableCacheEntry
- DistributedTableList -> CitusTableList
- isDistributedTable -> isCitusTable
- InsertSelectIntoDistributedTable -> InsertSelectIntoCitusTable
- ExtractFirstDistributedTableId -> ExtractFirstCitusTableId
2020-03-06 18:57:55 +00:00
Onur Tirtir 88bfd2e4b7 refactor around local group id checks
Mostyl optimizes the calls made to GetLocalGroupId and refactors
its usages
2020-03-05 20:20:41 +03:00
SaitTalhaNisanci a75436a54b
refactor CoordinatedTransactionCallback (#3571) 2020-03-05 18:36:12 +03:00
Philip Dubé 20abc4d2b5
Replace foreach with foreach_ptr/foreach_oid (#3544) 2020-02-27 16:54:49 +01:00
Jelte Fennema 685b54b3de
Semmle: Check for NULL in some places where it might occur (#3509)
Semmle reported quite some places where we use a value that could be NULL. Most of these are not actually a real issue, but better to be on the safe side with these things and make the static analysis happy.
2020-02-27 10:45:29 +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
Jelte Fennema 8de8b62669 Convert unsafe APIs to safe ones 2020-02-25 15:39:27 +01:00
Onur Tirtir 926a1a61b9 change "relation" with "table" in error messages related with foreign keys on reference tables 2020-02-20 09:58:47 +03:00
Onur Tirtir 001089783c Fix null relation name issue in CheckConflictingRelationAccesses 2020-02-19 19:10:35 +03:00
Jelte Fennema 3f7c5a5cf6
Semmle: Fix possible infite loops caused by overflow (#3503)
Comparison between differently sized integers in loop conditions can cause
infinite loops. This can happen when doing something like this:

```c
int64 very_big = MAX_INT32 + 1;
for (int32 i = 0; i < very_big; i++) {
    // do something
}
// never reached because i overflows before it can reach the value of very_big
```
2020-02-17 14:35:10 +01:00
Philip Dubé 3a906b8210 Fix typos noticed while reading through code trying to understand HAVING 2020-02-11 19:55:10 +00:00
Önder Kalacı ef7d1ea91d
Locally execute queries that don't need any data access (#3410)
* Update shardPlacement->nodeId to uint

As the source of the shardPlacement->nodeId is always workerNode->nodeId,
and that is uint32.

We had this hack because of: 0ea4e52df5 (r266421409)

And, that is gone with: 90056f7d3c (diff-c532177d74c72d3f0e7cd10e448ab3c6L1123)

So, we're safe to do it now.

* Relax the restrictions on using the local execution

Previously, whenever any local execution happens, we disabled further
commands to do any remote queries. The basic motivation for doing that
is to prevent any accesses in the same transaction block to access the
same placements over multiple sessions: one is local session the other
is remote session to the same placement.

However, the current implementation does not distinguish local accesses
being to a placement or not. For example, we could have local accesses
that only touches intermediate results. In that case, we should not
implement the same restrictions as they become useless.

So, this is a pre-requisite for executing the intermediate result only
queries locally.

* Update the error messages

As the underlying implementation has changed, reflect it in the error
messages.

* Keep track of connections to local node

With this commit, we're adding infrastructure to track if any connection
to the same local host is done or not.

The main motivation for doing this is that we've previously were more
conservative about not choosing local execution. Simply, we disallowed
local execution if any connection to any remote node is done. However,
if we want to use local execution for intermediate result only queries,
this'd be annoying because we expect all queries to touch remote node
before the final query.

Note that this approach is still limiting in Citus MX case, but for now
we can ignore that.

* Formalize the concept of Local Node

Also some minor refactoring while creating the dummy placement

* Write intermediate results locally when the results are only needed locally

Before this commit, Citus used to always broadcast all the intermediate
results to remote nodes. However, it is possible to skip pushing
the results to remote nodes always.

There are two notable cases for doing that:

   (a) When the query consists of only intermediate results
   (b) When the query is a zero shard query

In both of the above cases, we don't need to access any data on the shards. So,
it is a valuable optimization to skip pushing the results to remote nodes.

The pattern mentioned in (a) is actually a common patterns that Citus users
use in practice. For example, if you have the following query:

WITH cte_1 AS (...), cte_2 AS (....), ... cte_n (...)
SELECT ... FROM cte_1 JOIN cte_2 .... JOIN cte_n ...;

The final query could be operating only on intermediate results. With this patch,
the intermediate results of the ctes are not unnecessarily pushed to remote
nodes.

* Add specific regression tests

As there are edge cases in Citus MX and with round-robin policy,
use the same queries on those cases as well.

* Fix failure tests

By forcing not to use local execution for intermediate results since
all the tests expects the results to be pushed remotely.

* Fix flaky test

* Apply code-review feedback

Mostly style changes

* Limit the max value of pg_dist_node_seq to reserve for internal use
2020-01-23 18:28:34 +01:00
Hadi Moshayedi 44a2aede16 Don't start a coordinated transaction on workers.
Otherwise transaction hooks of Citus kick in and might cause unwanted errors.
2020-01-16 23:24:52 -08:00
Hadi Moshayedi e1e383cb59 Don't override xact id assigned by coordinator on workers.
We might need to send commands from workers to other workers. In
these cases we shouldn't override the xact id assigned by coordinator,
or otherwise we won't read the consistent set of result files
accross the nodes.
2020-01-09 11:09:11 -08: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
Nils Dijk bfc3d2eb90
make sure to correctly decrement ExecutorLevel (#3311)
DESCRIPTION: Fix counter that keeps track of internal depth in executor

While reviewing #3302 I ran into the `ExecutorLevel` variable which used a variable to keep the original value to restore on successful exit. I haven't explored the full space and if it is possible to get into an inconsistent state. However using `PG_TRY`/`PG_CATCH` seems generally more correct.

Given very bad things will happen if this level is not reset, I kept the failsafe of setting the variiable back to 0 on the `XactCallback` but I did add an assert to treat it as a developer bug.
2019-12-16 20:50:13 +01:00
Marco Slot 5f656e22db Fix issue in IsMultiStatementTransaction detection 2019-12-16 17:01:43 +01:00
SaitTalhaNisanci 2829c601dd
replace Begin words in coordinated transactions with use (#3293) 2019-12-16 10:40:31 +03:00
SaitTalhaNisanci d10f97998c rename REMOTE_TRANS_INVALID to REMOTE_TRANS_NOT_STARTED 2019-12-11 15:24:18 +03:00
Hadi Moshayedi d28beb3711 Detect SQL UDF Calls. 2019-12-05 14:31:05 -08:00
Marco Slot 16d1ad3666 Remove distinction between SQL_TASK and ROUTER_TASK 2019-11-29 05:58:29 +01: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
Philip Dubé 261a9de42d Fix typos:
VAR_SET_VALUE_KIND -> VAR_SET_VALUE kind
beginnig -> beginning
plannig -> planning
the the -> the
er then -> er than
2019-11-25 23:24:13 +00:00
Jelte Fennema 1d8dde232f
Automatically convert useless declarations using regex replace (#3181)
* Add declaration removal to CI

* Convert declarations
2019-11-21 13:47:29 +01:00
Hadi Moshayedi 15af1637aa Replicate reference tables to coordinator. 2019-11-15 05:50:19 -08:00
SaitTalhaNisanci b9b7fd7660
add IsLoggableLevel utility function (#3149)
* add IsLoggableLevel utility function

* add function comment for IsLoggableLevel

* put ApplyLogRedaction to logutils
2019-11-15 14:59:13 +03:00
Philip Dubé e8ecbbfcb3 Escape transaction names 2019-11-08 21:23:01 +00:00
SaitTalhaNisanci 02b359623f
remove duplicate code in citus_dist_stat_activity (#3165) 2019-11-08 15:41:32 +03:00
Önder Kalacı 960cd02c67
Remove real time router executors (#3142)
* Remove unused executor codes

All of the codes of real-time executor. Some functions
in router executor still remains there because there
are common functions. We'll move them to accurate places
in the follow-up commits.

* Move GUCs to transaction mngnt and remove unused struct

* Update test output

* Get rid of references of real-time executor from code

* Warn if real-time executor is picked

* Remove lots of unused connection codes

* Removed unused code for connection restrictions

Real-time and router executors cannot handle re-using of the existing
connections within a transaction block.

Adaptive executor and COPY can re-use the connections. So, there is no
reason to keep the code around for applying the restrictions in the
placement connection logic.
2019-11-05 12:48:10 +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
Hadi Moshayedi ae915493e6 Don't send metadata commands to not-synced workers.
Otherwise some of the dependencies might not exist yet and
commands will error out.
2019-10-03 16:52:25 -07:00
Marco Slot 2868e02a3d Implement SELECT function call delegation.
When a function is marked as colocated with a distributed table,
we try delegating queries of kind "SELECT func(...)" to workers.

We currently only support this simple form, and don't delegate
forms like "SELECT f1(...), f2(...)", "SELECT f1(...) FROM ...",
or function calls inside transactions.

As a side effect, we also fix the transactional semantics of DO blocks.
Previously we didn't consider a DO block a multi-statement transaction.
Now we do.

Co-authored-by: Marco Slot <marco@citusdata.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
2019-09-27 09:13:25 -07: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
Hadi Moshayedi 76f3933b05 Add metadatasynced, and sync on master_update_node()
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
2019-09-18 09:32:54 -07: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 0b0c779c77 Introduce the concept of Local Execution
/*
 * local_executor.c
 *
 * The scope of the local execution is locally executing the queries on the
 * shards. In other words, local execution does not deal with any local tables
 * that are not shards on the node that the query is being executed. In that sense,
 * the local executor is only triggered if the node has both the metadata and the
 * shards (e.g., only Citus MX worker nodes).
 *
 * The goal of the local execution is to skip the unnecessary network round-trip
 * happening on the node itself. Instead, identify the locally executable tasks and
 * simply call PostgreSQL's planner and executor.
 *
 * The local executor is an extension of the adaptive executor. So, the executor uses
 * adaptive executor's custom scan nodes.
 *
 * One thing to note that Citus MX is only supported with replication factor = 1, so
 * keep that in mind while continuing the comments below.
 *
 * On the high level, there are 3 slightly different ways of utilizing local execution:
 *
 * (1) Execution of local single shard queries of a distributed table
 *
 *      This is the simplest case. The executor kicks at the start of the adaptive
 *      executor, and since the query is only a single task the execution finishes
 *      without going to the network at all.
 *
 *      Even if there is a transaction block (or recursively planned CTEs), as long
 *      as the queries hit the shards on the same, the local execution will kick in.
 *
 * (2) Execution of local single queries and remote multi-shard queries
 *
 *      The rule is simple. If a transaction block starts with a local query execution,
 *      all the other queries in the same transaction block that touch any local shard
 *      have to use the local execution. Although this sounds restrictive, we prefer to
 *      implement in this way, otherwise we'd end-up with as complex scenarious as we
 *      have in the connection managements due to foreign keys.
 *
 *      See the following example:
 *      BEGIN;
 *          -- assume that the query is executed locally
 *          SELECT count(*) FROM test WHERE key = 1;
 *
 *          -- at this point, all the shards that reside on the
 *          -- node is executed locally one-by-one. After those finishes
 *          -- the remaining tasks are handled by adaptive executor
 *          SELECT count(*) FROM test;
 *
 *
 * (3) Modifications of reference tables
 *
 *		Modifications to reference tables have to be executed on all nodes. So, after the
 *		local execution, the adaptive executor keeps continuing the execution on the other
 *		nodes.
 *
 *		Note that for read-only queries, after the local execution, there is no need to
 *		kick in adaptive executor.
 *
 *  There are also few limitations/trade-offs that is worth mentioning. First, the
 *  local execution on multiple shards might be slow because the execution has to
 *  happen one task at a time (e.g., no parallelism). Second, if a transaction
 *  block/CTE starts with a multi-shard command, we do not use local query execution
 *  since local execution is sequential. Basically, we do not want to lose parallelism
 *  across local tasks by switching to local execution. Third, the local execution
 *  currently only supports queries. In other words, any utility commands like TRUNCATE,
 *  fails if the command is executed after a local execution inside a transaction block.
 *  Forth, the local execution cannot be mixed with the executors other than adaptive,
 *  namely task-tracker, real-time and router executors. Finally, related with the
 *  previous item, COPY command cannot be mixed with local execution in a transaction.
 *  The implication of that any part of INSERT..SELECT via coordinator cannot happen
 *  via the local execution.
 */
2019-09-12 11:51:25 +02:00
Philip Dubé 32ef459025 backend_data.c: include max_wal_senders in calculating maxBackend, matches changes in pg12's InitializeMaxBackends 2019-08-28 21:24:33 +00:00
Jelte Fennema cbecf97c84
Move tuplestore setup to a helper function (#2898)
* Add tuplestore helpers

* More detailed error messages in tuplestore

* Add CreateTupleDescCopy to SetupTuplestore

* Use new SetupTuplestore helper function

* Remove unnecessary copy

* Remove comment about undefined behaviour
2019-08-27 09:11:08 +02:00
Philip Dubé 68c4b71f93 Fix up includes with pg12 changes 2019-08-22 18:56:21 +00:00
Hadi Moshayedi c582eb89c8 Add some missing locks. 2019-08-15 12:34:31 -07:00
Philip Dubé f4e513b3d4 Introduce citus.single_shard_commit_protocol for if users want 1PC on writes to replicas 2019-08-15 18:49:40 +00:00
Onder Kalaci 060ac11476 Do not record relation accessess unnecessarily
Before this commit, we've recorded the relation accesses in 3 different
places
    - FindPlacementListConnection         -- applies all executor in tx block
    - StartPlacementExecutionOnSession()  -- adaptive executor only
    - StartPlacementListConnection()      -- router/real-time only

This is different than Citus 8.2, and could lead to query execution times
increase considerably on multi-shard commands in transaction block
that are on partitioned tables.

Benchmarks:

```
1+8 c5.4xlarge cluster

Empty distributed partitioned table with 365 partitions: https://gist.github.com/onderkalaci/1edace4ed6bd6f061c8a15594865bb51#file-partitions_365-sql

./pgbench -f /tmp/multi_shard.sql -c10 -j10 -P 1 -T 120 postgres://citus:w3r6KLJpv3mxe9E-NIUeJw@c.fy5fkjcv45vcepaogqcaskmmkee.db.citusdata.com:5432/citus?sslmode=require

cat  /tmp/multi_shard.sql
BEGIN;
	DELETE FROM collections_list;
	DELETE FROM collections_list;
	DELETE FROM collections_list;
COMMIT;
cat  /tmp/single_shard.sql
BEGIN;
	DELETE FROM collections_list WHERE key = :aid;
	DELETE FROM collections_list WHERE key = :aid;
	DELETE FROM collections_list WHERE key = :aid;
COMMIT;

cat  /tmp/mix.sql
BEGIN;
	DELETE FROM collections_list WHERE key = :aid;
	DELETE FROM collections_list WHERE key = :aid;
	DELETE FROM collections_list WHERE key = :aid;

	DELETE FROM collections_list;
	DELETE FROM collections_list;
	DELETE FROM collections_list;
COMMIT;
```

The table shows `latency average` of pgbench runs explained above, so we have a pretty solid improvement even over 8.2.2.

| Test  | Citus 8.2.2  |  Citus 8.3.1   | Citus 8.3.2 (this branch)  | Citus 8.3.1 (FKEYs disabled via GUC)  |
| ------------- | ------------- | ------------- |------------- | ------------- |
|multi_shard |  2370.083 ms  |3605.040 ms |1324.094 ms |1247.255 ms  |
| single_shard  | 85.338 ms  |120.934 ms  |73.216 ms  | 78.765 ms |
| mix  | 2434.459 ms | 3727.080 ms  |1306.456 ms  | 1280.326 ms |
2019-08-08 18:42:08 +02:00
Önder Kalacı 40da78c6fd
Introduce the adaptive executor (#2798)
With this commit, we're introducing the Adaptive Executor. 


The commit message consists of two distinct sections. The first part explains
how the executor works. The second part consists of the commit messages of
the individual smaller commits that resulted in this commit. The readers
can search for the each of the smaller commit messages on 
https://github.com/citusdata/citus and can learn more about the history
of the change.

/*-------------------------------------------------------------------------
 *
 * adaptive_executor.c
 *
 * The adaptive executor executes a list of tasks (queries on shards) over
 * a connection pool per worker node. The results of the queries, if any,
 * are written to a tuple store.
 *
 * The concepts in the executor are modelled in a set of structs:
 *
 * - DistributedExecution:
 *     Execution of a Task list over a set of WorkerPools.
 * - WorkerPool
 *     Pool of WorkerSessions for the same worker which opportunistically
 *     executes "unassigned" tasks from a queue.
 * - WorkerSession:
 *     Connection to a worker that is used to execute "assigned" tasks
 *     from a queue and may execute unasssigned tasks from the WorkerPool.
 * - ShardCommandExecution:
 *     Execution of a Task across a list of placements.
 * - TaskPlacementExecution:
 *     Execution of a Task on a specific placement.
 *     Used in the WorkerPool and WorkerSession queues.
 *
 * Every connection pool (WorkerPool) and every connection (WorkerSession)
 * have a queue of tasks that are ready to execute (readyTaskQueue) and a
 * queue/set of pending tasks that may become ready later in the execution
 * (pendingTaskQueue). The tasks are wrapped in a ShardCommandExecution,
 * which keeps track of the state of execution and is referenced from a
 * TaskPlacementExecution, which is the data structure that is actually
 * added to the queues and describes the state of the execution of a task
 * on a particular worker node.
 *
 * When the task list is part of a bigger distributed transaction, the
 * shards that are accessed or modified by the task may have already been
 * accessed earlier in the transaction. We need to make sure we use the
 * same connection since it may hold relevant locks or have uncommitted
 * writes. In that case we "assign" the task to a connection by adding
 * it to the task queue of specific connection (in
 * AssignTasksToConnections). Otherwise we consider the task unassigned
 * and add it to the task queue of a worker pool, which means that it
 * can be executed over any connection in the pool.
 *
 * A task may be executed on multiple placements in case of a reference
 * table or a replicated distributed table. Depending on the type of
 * task, it may not be ready to be executed on a worker node immediately.
 * For instance, INSERTs on a reference table are executed serially across
 * placements to avoid deadlocks when concurrent INSERTs take conflicting
 * locks. At the beginning, only the "first" placement is ready to execute
 * and therefore added to the readyTaskQueue in the pool or connection.
 * The remaining placements are added to the pendingTaskQueue. Once
 * execution on the first placement is done the second placement moves
 * from pendingTaskQueue to readyTaskQueue. The same approach is used to
 * fail over read-only tasks to another placement.
 *
 * Once all the tasks are added to a queue, the main loop in
 * RunDistributedExecution repeatedly does the following:
 *
 * For each pool:
 * - ManageWorkPool evaluates whether to open additional connections
 *   based on the number unassigned tasks that are ready to execute
 *   and the targetPoolSize of the execution.
 *
 * Poll all connections:
 * - We use a WaitEventSet that contains all (non-failed) connections
 *   and is rebuilt whenever the set of active connections or any of
 *   their wait flags change.
 *
 *   We almost always check for WL_SOCKET_READABLE because a session
 *   can emit notices at any time during execution, but it will only
 *   wake up WaitEventSetWait when there are actual bytes to read.
 *
 *   We check for WL_SOCKET_WRITEABLE just after sending bytes in case
 *   there is not enough space in the TCP buffer. Since a socket is
 *   almost always writable we also use WL_SOCKET_WRITEABLE as a
 *   mechanism to wake up WaitEventSetWait for non-I/O events, e.g.
 *   when a task moves from pending to ready.
 *
 * For each connection that is ready:
 * - ConnectionStateMachine handles connection establishment and failure
 *   as well as command execution via TransactionStateMachine.
 *
 * When a connection is ready to execute a new task, it first checks its
 * own readyTaskQueue and otherwise takes a task from the worker pool's
 * readyTaskQueue (on a first-come-first-serve basis).
 *
 * In cases where the tasks finish quickly (e.g. <1ms), a single
 * connection will often be sufficient to finish all tasks. It is
 * therefore not necessary that all connections are established
 * successfully or open a transaction (which may be blocked by an
 * intermediate pgbouncer in transaction pooling mode). It is therefore
 * essential that we take a task from the queue only after opening a
 * transaction block.
 *
 * When a command on a worker finishes or the connection is lost, we call
 * PlacementExecutionDone, which then updates the state of the task
 * based on whether we need to run it on other placements. When a
 * connection fails or all connections to a worker fail, we also call
 * PlacementExecutionDone for all queued tasks to try the next placement
 * and, if necessary, mark shard placements as inactive. If a task fails
 * to execute on all placements, the execution fails and the distributed
 * transaction rolls back.
 *
 * For multi-row INSERTs, tasks are executed sequentially by
 * SequentialRunDistributedExecution instead of in parallel, which allows
 * a high degree of concurrency without high risk of deadlocks.
 * Conversely, multi-row UPDATE/DELETE/DDL commands take aggressive locks
 * which forbids concurrency, but allows parallelism without high risk
 * of deadlocks. Note that this is unrelated to SEQUENTIAL_CONNECTION,
 * which indicates that we should use at most one connection per node, but
 * can run tasks in parallel across nodes. This is used when there are
 * writes to a reference table that has foreign keys from a distributed
 * table.
 *
 * Execution finishes when all tasks are done, the query errors out, or
 * the user cancels the query.
 *
 *-------------------------------------------------------------------------
 */



All the commits involved here:
* Initial unified executor prototype

* Latest changes

* Fix rebase conflicts to master branch

* Add missing variable for assertion

* Ensure that master_modify_multiple_shards() returns the affectedTupleCount

* Adjust intermediate result sizes

The real-time executor uses COPY command to get the results
from the worker nodes. Unified executor avoids that which
results in less data transfer. Simply adjust the tests to lower
sizes.

* Force one connection per placement (or co-located placements) when requested

The existing executors (real-time and router) always open 1 connection per
placement when parallel execution is requested.

That might be useful under certain circumstances:

(a) User wants to utilize as much as CPUs on the workers per
distributed query
(b) User has a transaction block which involves COPY command

Also, lots of regression tests rely on this execution semantics.
So, we'd enable few of the tests with this change as well.

* For parameters to be resolved before using them

For the details, see PostgreSQL's copyParamList()

* Unified executor sorts the returning output

* Ensure that unified executor doesn't ignore sequential execution of DDLJob's

Certain DDL commands, mainly creating foreign keys to reference tables,
should be executed sequentially. Otherwise, we'd end up with a self
distributed deadlock.

To overcome this situaiton, we set a flag `DDLJob->executeSequentially`
and execute it sequentially. Note that we have to do this because
the command might not be called within a transaction block, and
we cannot call `SetLocalMultiShardModifyModeToSequential()`.

This fixes at least two test: multi_insert_select_on_conflit.sql and
multi_foreign_key.sql

Also, I wouldn't mind scattering local `targetPoolSize` variables within
the code. The reason is that we'll soon have a GUC (or a global
variable based on a GUC) that'd set the pool size. In that case, we'd
simply replace `targetPoolSize` with the global variables.

* Fix 2PC conditions for DDL tasks

* Improve closing connections that are not fully established in unified execution

* Support foreign keys to reference tables in unified executor

The idea for supporting foreign keys to reference tables is simple:
Keep track of the relation accesses within a transaction block.
    - If a parallel access happens on a distributed table which
      has a foreign key to a reference table, one cannot modify
      the reference table in the same transaction. Otherwise,
      we're very likely to end-up with a self-distributed deadlock.
    - If an access to a reference table happens, and then a parallel
      access to a distributed table (which has a fkey to the reference
      table) happens, we switch to sequential mode.

Unified executor misses the function calls that marks the relation
accesses during the execution. Thus, simply add the necessary calls
and let the logic kick in.

* Make sure to close the failed connections after the execution

* Improve comments

* Fix savepoints in unified executor.

* Rebuild the WaitEventSet only when necessary

* Unclaim connections on all errors.

* Improve failure handling for unified executor

   - Implement the notion of errorOnAnyFailure. This is similar to
     Critical Connections that the connection managament APIs provide
   - If the nodes inside a modifying transaction expand, activate 2PC
   - Fix few bugs related to wait event sets
   - Mark placement INACTIVE during the execution as much as possible
     as opposed to we do in the COMMIT handler
   - Fix few bugs related to scheduling next placement executions
   - Improve decision on when to use 2PC

Improve the logic to start a transaction block for distributed transactions

- Make sure that only reference table modifications are always
  executed with distributed transactions
- Make sure that stored procedures and functions are executed
  with distributed transactions

* Move waitEventSet to DistributedExecution

This could also be local to RunDistributedExecution(), but in that case
we had to mark it as "volatile" to avoid PG_TRY()/PG_CATCH() issues, and
cast it to non-volatile when doing WaitEventSetFree(). We thought that
would make code a bit harder to read than making this non-local, so we
move it here. See comments for PG_TRY() in postgres/src/include/elog.h
and "man 3 siglongjmp" for more context.

* Fix multi_insert_select test outputs

Two things:
   1) One complex transaction block is now supported. Simply update
      the test output
   2) Due to dynamic nature of the unified executor, the orders of
      the errors coming from the shards might change (e.g., all of
      the queries on the shards would fail, but which one appears
      on the error message?). To fix that, we simply added it to
      our shardId normalization tool which happens just before diff.

* Fix subeury_and_cte test

The error message is updated from:
	failed to execute task
To:
        more than one row returned by a subquery or an expression

which is a lot clearer to the user.

* Fix intermediate_results test outputs

Simply update the error message from:
	could not receive query results
to
	result "squares" does not exist

which makes a lot more sense.

* Fix multi_function_in_join test

The error messages update from:
     Failed to execute task XXX
To:
     function f(..) does not exist

* Fix multi_query_directory_cleanup test

The unified executor does not create any intermediate files.

* Fix with_transactions test

A test case that just started to work fine

* Fix multi_router_planner test outputs

The error message is update from:
	Could not receive query results
To:
	Relation does not exists

which is a lot more clearer for the users

* Fix multi_router_planner_fast_path test

The error message is update from:
	Could not receive query results
To:
	Relation does not exists

which is a lot more clearer for the users

* Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block

* Fix ordering in isolation_multi_shard_modify_vs_all

* Add executor locks to unified executor

* Make sure to allocate enought WaitEvents

The previous code was missing the waitEvents for the latch and
postmaster death.

* Fix rebase conflicts for master rebase

* Make sure that TRUNCATE relies on unified executor

* Implement true sequential execution for multi-row INSERTS

Execute the individual tasks executed one by one. Note that this is different than
MultiShardConnectionType == SEQUENTIAL_CONNECTION case (e.g., sequential execution
mode). In that case, running the tasks across the nodes in parallel is acceptable
and implemented in that way.

However, the executions that are qualified here would perform poorly if the
tasks across the workers are executed in parallel. We currently qualify only
one class of distributed queries here, multi-row INSERTs. If we do not enforce
true sequential execution, concurrent multi-row upserts could easily form
a distributed deadlock when the upserts touch the same rows.

* Remove SESSION_LIFESPAN flag in unified_executor

* Apply failure test updates

We've changed the failure behaviour a bit, and also the error messages
that show up to the user. This PR covers majority of the updates.

* Unified executor honors citus.node_connection_timeout

With this commit, unified executor errors out if even
a single connection cannot be established within
citus.node_connection_timeout.

And, as a side effect this fixes failure_connection_establishment
test.

* Properly increment/decrement pool size variables

Before this commit, the idle and active connection
counts were not properly calculated.

* insert_select_executor goes through unified executor.

* Add missing file for task tracker

* Modify ExecuteTaskListExtended()'s signature

* Sort output of INSERT ... SELECT ... RETURNING

* Take partition locks correctly in unified executor

* Alternative implementation for force_max_query_parallelization

* Fix compile warnings in unified executor

* Fix style issues

* Decrement idleConnectionCount when idle connection is lost

* Always rebuild the wait event sets

In the previous implementation, on waitFlag changes, we were only
modifying the wait events. However, we've realized that it might
be an over optimization since (a) we couldn't see any performance
benefits (b) we see some errors on failures and because of (a)
we prefer to disable it now.

* Make sure to allocate enough sized waitEventSet

With multi-row INSERTs, we might have more sessions than
task*workerCount after few calls of RunDistributedExecution()
because the previous sessions would also be alive.

Instead, re-allocate events when the connectino set changes.

* Implement SELECT FOR UPDATE on reference tables

On master branch, we do two extra things on SELECT FOR UPDATE
queries on reference tables:
   - Acquire executor locks
   - Execute the query on all replicas

With this commit, we're implementing the same logic on the
new executor.

* SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled

Otherwise, users would be very confused and their logic is very likely
to break.

* Fix build error

* Fix the newConnectionCount calculation in ManageWorkerPool

* Fix rebase conflicts

* Fix minor test output differences

* Fix citus indent

* Remove duplicate sorts that is added with rebase

* Create distributed table via executor

* Fix wait flags in CheckConnectionReady

* failure_savepoints output for unified executor.

* failure_vacuum output (pg 10) for unified executor.

* Fix WaitEventSetWait timeout in unified executor

* Stabilize failure_truncate test output

* Add an ORDER BY to multi_upsert

* Fix regression test outputs after rebase to master

* Add executor.c comment

* Rename executor.c to adaptive_executor.c

* Do not schedule tasks if the failed placement is not ready to execute

Before the commit, we were blindly scheduling the next placement executions
even if the failed placement is not on the ready queue. Now, we're ensuring
that if failed placement execution is on a failed pool or session where the
execution is on the pendingQueue, we do not schedule the next task. Because
the other placement execution should be already running.

* Implement a proper custom scan node for adaptive executor

- Switch between the executors, add GUC to set the pool size
- Add non-adaptive regression test suites
- Enable CIRCLE CI for non-adaptive tests
- Adjust test output files

* Add slow start interval to the executor

* Expose max_cached_connection_per_worker to user

* Do not start slow when there are cached connections

* Consider ExecutorSlowStartInterval in NextEventTimeout

* Fix memory issues with ReceiveResults().

* Disable executor via TaskExecutorType

* Make sure to execute the tests with the other executor

* Use task_executor_type to enable-disable adaptive executor

* Remove useless code

* Adjust the regression tests

* Add slow start regression test

* Rebase to master

* Fix test failures in adaptive executor.

* Rebase to master - 2

* Improve comments & debug messages

* Set force_max_query_parallelization in isolation_citus_dist_activity

* Force max parallelization for creating shards when asked to use exclusive connection.

* Adjust the default pool size

* Expand description of max_adaptive_executor_pool_size GUC

* Update warnings in FinishRemoteTransactionCommit()

* Improve session clean up at the end of execution

Explicitly list all the states that the execution might end,
otherwise warn.

* Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all

* Add more ORDER BYs to multi_mx_partitioning
2019-06-28 14:04:40 +02:00
Jason Petersen d4e1172247 Implement propagation of SET LOCAL commands
Adds support for propagation of SET LOCAL commands to all workers
involved in a query. For now, SET SESSION (i.e. plain SET) is not
supported whatsoever, though this code is intended as somewhat of a
base for implementing such support in the future.

As SET LOCAL modifications are scoped to the body of a BEGIN/END xact
block, queries wishing to use SET LOCAL propagation must be within such
a block. In addition, subsequent modifications after e.g. any SAVEPOINT
or ROLLBACK statements will correspondingly push or pop variable mod-
ifications onto an internal stack such that the behavior of changed
values across the cluster will be identical to such behavior on e.g.
single-node PostgreSQL (or equivalently, what values are visible to
the end user by running SHOW on such variables on the coordinator).

If nodes enter the set of participants at some point after SET LOCAL
modifications (or SAVEPOINT, ROLLBACK, etc.) have occurred, the SET
variable state is eagerly propagated to them upon their entrance (this
is identical to, and indeed just augments, the existing logic for the
propagation of the SAVEPOINT "stack").

A new GUC (citus.propagate_set_commands) has been added to control this
behavior. Though the code suggests the valid settings are 'none', 'local',
'session', and 'all', only 'none' (the default) and 'local' are presently
implemented: attempting to use other values will result in an error.
2019-06-20 16:15:43 -07:00
Jason Petersen 1dec6c5163 Change BeginCoordinatedTransaction to internal linkage
It's only ever called from a single file, so having it be extern didn't
make a whole lot of sense.
2019-06-20 13:44:06 -07:00
Hadi Moshayedi 4bbae02778 Make COPY compatible with unified executor. 2019-06-20 19:53:40 +02:00
Hadi Moshayedi 83f6c7dab4 Fix subxact release crash 2019-06-19 17:43:10 +02:00
Marco Slot bb3a96eacb Cache a configurable number of connections at xact end 2019-05-29 13:24:31 +02:00
Onder Kalaci 495b6e9b62 Refactor Parallel Relation Access Recording
Instead of scattering the code around, we move all the
logic into a single function.

This will help supporting foreign keys to reference tables
in the unified executor with a single line of change, just
calling this function.
2019-05-02 18:12:33 +03:00
Hadi Moshayedi aafd22dffa Fix savepoint rollback for INSERT INTO ... SELECT. 2019-05-01 09:33:43 -07:00
Hadi Moshayedi b69a762e0b Fix savepoint rollback after multi-shard update failure. 2019-05-01 09:33:43 -07:00
Nils Dijk feaac69769
Implementation for asycn FinishConnectionListEstablishment (#2584) 2019-03-22 17:30:42 +01:00
Marco Slot f2abf2b8e5 Functions are treated as transaction blocks 2019-03-15 16:34:08 -06:00
Hadi Moshayedi f4d3b94e22
Fix some of the casts for groupId (#2609)
A small change which partially addresses #2608.
2019-03-05 12:06:44 -08:00
Onder Kalaci f706772b2f Round-robin task assignment policy relies on local transaction id
Before this commit, round-robin task assignment policy was relying
on the taskId. Thus, even inside a transaction, the tasks were
assigned to different nodes. This was especially problematic
while reading from reference tables within transaction blocks.
Because, we had to expand the distributed transaction to many
nodes that are not necessarily already in the distributed transaction.
2019-02-22 19:26:38 +03:00
Jason Petersen 339e6e661e
Remove 9.6 (#2554)
Removes support and code for PostgreSQL 9.6

cr: @velioglu
2019-01-16 13:11:24 -07:00
Murat Tuncer ec36030fae Move functions calls that can fail to outside of spinlock
We had recently fixed a spinlock issue due to functions
failing, but spinlock is not being released.

This is the continuation of that work to eliminate possible
regression of the issue. Function calls that are moved out of
spinlock scope are macros and plain type casting. However,
depending on the configuration they have an alternate implementation
in PG source that performs memory allocation.

This commit moves last bit of codes to out of spinlock for completion purposes.
2019-01-03 15:59:56 +03:00
Murat Tuncer 9671bc3cbb Make sure spinlock is not left unreleased when an exception is thrown
A spinlock is not released when an exception is thrown after
spinlock is acquired. This has caused infinite wait and eventual
crash in maintenance daemon.

This work moves the code than can fail to the outside of spinlock
scope so that in the case of failure spinlock is not left locked
since it was not locked in the first place.
2018-12-24 15:47:21 +03:00
Marco Slot 3ff2b47366 Restrict visibility of get_*_active_transactions functions to pg_monitor 2018-12-19 18:32:42 +01:00
Dimitri Fontaine d1b182de7d Replace calls to unsafe functions like memcpy and sscanf
In answer to a security audit, we double check buffer sizes and avoid
known-dangerous operations such as sscanf.
2018-12-04 08:54:43 +01:00
Onder Kalaci 621ccf3946 Ensure to use initialized MaxBackends
Postgresql loads shared libraries before calculating MaxBackends.
However, Citus relies on MaxBackends being set. Thus, with this
commit we use the same steps to calculate MaxBackends while
Citus is being loaded (e.g., PG_Init is called).

Note that this is safe since all the elements that are used to
calculate MaxBackends are PGC_POSTMASTER gucs and a constant
value.
2018-12-03 13:25:51 +03:00
Jason Petersen 9fb951c312
Fix user-facing typos
Lintian found these (presumably by looking in the text section and
running them through e.g. aspell).
2018-10-09 16:54:03 -07:00
Onder Kalaci 73696a03e4 Make sure not to leak intermediate result folders on the workers 2018-10-09 22:47:56 +03:00
velioglu 512d23934f Show router modify,select and real-time queries on MX views 2018-10-02 13:59:38 +03:00
Murat Tuncer 653c7e4ae0 Fix memory leak in FinishRemoteTransactionPrepare 2018-09-28 11:13:21 +03:00
Onder Kalaci 4cae856846 Relax assertion on transaction abort on PREPARE step
In case a failure happens when a transaction is failed on PREPARE,
we used to hit an assertion for ensuring there is no pending
activity on the connection. However, that's not true after the
changes in #2031. Thus, we've replaced the assertion with a more
generic function call to consume any pending activity, if exists.
2018-09-17 18:09:16 +03:00
Onder Kalaci a94184fff8 Prevent overflow of memory accesses during deadlock detection
In the distributed deadlock detection design, we concluded that prepared transactions
cannot be part of a distributed deadlock. The idea is that (a) when the transaction
is prepared it already acquires all the locks, so cannot be part of a deadlock
(b) even if some other processes blocked on the prepared transaction,  prepared transactions
 would eventually be committed (or rollbacked) and the system will continue operating.

With the above in mind, we probably had a mistake in terms of memory allocations. For each
backend initialized, we keep a `BackendData` struct. The bug we've introduced is that, we
assumed there would only be `MaxBackend` number of backends. However, `MaxBackends` doesn't
include the prepared transactions and axuliary processes. When you check Postgres' InitProcGlobal`
you'd see that `TotalProcs = MaxBackends + NUM_AUXILIARY_PROCS + max_prepared_xacts;`

This commit aligns with total procs processed with that.
2018-09-17 16:23:29 +03:00
velioglu d1f005daac Adds UDFs for testing MX functionalities with isolation tests 2018-09-12 07:04:16 +03:00
Onder Kalaci d657759c97 Views to Provide some insight about the distributed transactions on Citus MX
With this commit, we implement two views that are very similar
to pg_stat_activity, but showing queries that are involved in
distributed queries:

    - citus_dist_stat_activity: Shows all the distributed queries
    - citus_worker_stat_activity: Shows all the queries on the shards
                                  that are initiated by distributed queries.

Both views have the same columns in the outputs. In very basic terms, both of the views
are meant to provide some useful insights about the distributed
transactions within the cluster. As the names reveal, both views are similar to pg_stat_activity.
Also note that these views can be pretty useful on Citus MX clusters.

Note that when the views are queried from the worker nodes, they'd not show the distributed
transactions that are initiated from the coordinator node. The reason is that the worker
nodes do not know the host/port of the coordinator. Thus, it is advisable to query the
views from the coordinator.

If we bucket the columns that the views returns, we'd end up with the following:

- Hostnames and ports:
   - query_hostname, query_hostport: The node that the query is running
   - master_query_host_name, master_query_host_port: The node in the cluster
                                                   initiated the query.
    Note that for citus_dist_stat_activity view, the query_hostname-query_hostport
    is always the same with master_query_host_name-master_query_host_port. The
    distinction is mostly relevant for citus_worker_stat_activity. For example,
    on Citus MX, a users starts a transaction on Node-A, which starts worker
    transactions on Node-B and Node-C. In that case, the query hostnames would be
    Node-B and Node-C whereas the master_query_host_name would Node-A.

- Distributed transaction related things:
    This is mostly the process_id, distributed transactionId and distributed transaction
    number.

- pg_stat_activity columns:
    These two views get all the columns from pg_stat_activity. We're basically joining
    pg_stat_activity with get_all_active_transactions on process_id.
2018-09-10 21:33:27 +03: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 bf28dd0cff Do not recover wrong distributed transactions in MX 2018-09-07 09:52:46 +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
mehmet furkan şahin ef9f38b68d ApplyLogRedaction noop func is added 2018-08-17 14:48:54 -07:00
Nils Dijk 2a9d47e1a6
fix pg11 tests 2018-08-15 23:27:31 -06:00
Onder Kalaci 7fb529aab9 Some stylistic improvements in the foreign keys to reference table
changes.
2018-07-05 23:23:34 +03:00
Onder Kalaci d83be3a33f Enforce foreign key restrictions inside transaction blocks
When a hash distributed table have a foreign key to a reference
table, there are few restrictions we have to apply in order to
prevent distributed deadlocks or reading wrong results.

The necessity to apply the restrictions arise from cascading
nature of foreign keys. When a foreign key on a reference table
cascades to a distributed table, a single operation over a single
connection can acquire locks on multiple shards of the distributed
table. Thus, any parallel operation on that distributed table, in the
same transaction should not open parallel connections to the shards.
Otherwise, we'd either end-up with a self-distributed deadlock or
read wrong results.

As briefly described above, the restrictions that we apply is done
by tracking the distributed/reference relation accesses inside
transaction blocks, and act accordingly when necessary.

The two main rules are as follows:
   - Whenever a parallel distributed relation access conflicts
     with a consecutive reference relation access, Citus errors
     out
   - Whenever a reference relation access is followed by a
     conflicting parallel relation access, the execution mode
     is switched to sequential mode.

There are also some other notes to mention:
   - If the user does SET LOCAL citus.multi_shard_modify_mode
     TO 'sequential';, all the queries should simply work with
     using one connection per worker and sequentially executing
     the commands. That's obviously a slower approach than Citus'
     usual parallel execution. However, we've at least have a way
     to run all commands successfully.

   - If an unrelated parallel query executed on any distributed
     table, we cannot switch to sequential mode. Because, the essense
     of sequential mode is using one connection per worker. However,
     in the presence of a parallel connection, the connection manager
     picks those connections to execute the commands. That contradicts
     with our purpose, thus we error out.

   - COPY to a distributed table cannot be executed in sequential mode.
     Thus, if we switch to sequential mode and COPY is executed, the
     operation fails and there is currently no way of implementing that.
     Note that, when the local table is not empty and create_distributed_table
     is used, citus uses COPY internally. Thus, in those cases,
     create_distributed_table() will also fail.

   - There is a GUC called citus.enforce_foreign_key_restrictions
     to disable all the checks. We added that GUC since the restrictions
     we apply is sometimes a bit more restrictive than its necessary.
     The user might want to relax those. Similarly, if you don't have
     CASCADEing reference tables, you might consider disabling all the
     checks.
2018-07-03 17:05:55 +03:00
Onder Kalaci 7d0f7835e7 Improve relation accesses association to do less job 2018-06-25 18:40:40 +03:00
Onder Kalaci 8ccb8b679e Real-time executor marks multi shard relation accesses before opening connections 2018-06-25 18:40:31 +03:00
Onder Kalaci 21038f0d0e Make sure that inter-shard DDL commands are always covers both tables 2018-06-25 18:40:30 +03:00
Onder Kalaci 2f01894589 Track relation accesses using the connection management infrastructure 2018-06-25 18:40:30 +03:00
Marco Slot 0feb1f2eb1 Do not call CheckRemoteTransactionsHealth from commit handler 2018-06-14 23:33:07 +02:00
Marco Slot 4ab8e87090 Always throw errors on failure on critical connection in router executor 2018-06-14 23:33:07 +02:00
Brian Cloutier 9667ee5ac9 Alleviate OOM failures in COMMIT callback
Previously those failures caused us to crash, postgres abort()s when it
notices a failure in the COMMIT callback.
2018-05-15 16:39:33 -07:00