This change adds a script to programatically group all includes in a
specific order. The script was used as a one time invocation to group
and sort all includes throught our formatted code. The grouping is as
follows:
- System includes (eg. `#include<...>`)
- Postgres.h (eg. `#include "postgres.h"`)
- Toplevel imports from postgres, not contained in a directory (eg.
`#include "miscadmin.h"`)
- General postgres includes (eg . `#include "nodes/..."`)
- Toplevel citus includes, not contained in a directory (eg. `#include
"citus_verion.h"`)
- Columnar includes (eg. `#include "columnar/..."`)
- Distributed includes (eg. `#include "distributed/..."`)
Because it is quite hard to understand the difference between toplevel
citus includes and toplevel postgres includes it hardcodes the list of
toplevel citus includes. In the same manner it assumes anything not
prefixed with `columnar/` or `distributed/` as a postgres include.
The sorting/grouping is enforced by CI. Since we do so with our own
script there are not changes required in our uncrustify configuration.
DESCRIPTION: Adds citus.enable_schema_based_sharding GUC that allows
sharding the database based on schemas when enabled.
* Refactor the logic that automatically creates Citus managed tables
* Refactor CreateSingleShardTable() to allow specifying colocation id
instead
* Add support for schema-based-sharding via a GUC
### What this PR is about:
Add **citus.enable_schema_based_sharding GUC** to enable schema-based
sharding. Each schema created while this GUC is ON will be considered
as a tenant schema. Later on, regardless of whether the GUC is ON or
OFF, any table created in a tenant schema will be converted to a
single shard distributed table (without a shard key). All the tenant
tables that belong to a particular schema will be co-located with each
other and will have a shard count of 1.
We introduce a new metadata table --pg_dist_tenant_schema-- to do the
bookkeeping for tenant schemas:
```sql
psql> \d pg_dist_tenant_schema
Table "pg_catalog.pg_dist_tenant_schema"
┌───────────────┬─────────┬───────────┬──────────┬─────────┐
│ Column │ Type │ Collation │ Nullable │ Default │
├───────────────┼─────────┼───────────┼──────────┼─────────┤
│ schemaid │ oid │ │ not null │ │
│ colocationid │ integer │ │ not null │ │
└───────────────┴─────────┴───────────┴──────────┴─────────┘
Indexes:
"pg_dist_tenant_schema_pkey" PRIMARY KEY, btree (schemaid)
"pg_dist_tenant_schema_unique_colocationid_index" UNIQUE, btree (colocationid)
psql> table pg_dist_tenant_schema;
┌───────────┬───────────────┐
│ schemaid │ colocationid │
├───────────┼───────────────┤
│ 41963 │ 91 │
│ 41962 │ 90 │
└───────────┴───────────────┘
(2 rows)
```
Colocation id column of pg_dist_tenant_schema can never be NULL even
for the tenant schemas that don't have a tenant table yet. This is
because, we assign colocation ids to tenant schemas as soon as they
are created. That way, we can keep associating tenant schemas with
particular colocation groups even if all the tenant tables of a tenant
schema are dropped and recreated later on.
When a tenant schema is dropped, we delete the corresponding row from
pg_dist_tenant_schema. In that case, we delete the corresponding
colocation group from pg_dist_colocation as well.
### Future work for 12.0 release:
We're building schema-based sharding on top of the infrastructure that
adds support for creating distributed tables without a shard key
(https://github.com/citusdata/citus/pull/6867).
However, not all the operations that can be done on distributed tables
without a shard key necessarily make sense (in the same way) in the
context of schema-based sharding. For example, we need to think about
what happens if user attempts altering schema of a tenant table. We
will tackle such scenarios in a future PR.
We will also add a new UDF --citus.schema_tenant_set() or such-- to
allow users to use an existing schema as a tenant schema, and another
one --citus.schema_tenant_unset() or such-- to stop using a schema as
a tenant schema in future PRs.
* Remove if conditions with PG_VERSION_NUM < 13
* Remove server_above_twelve(&eleven) checks from tests
* Fix tests
* Remove pg12 and pg11 alternative test output files
* Remove pg12 specific normalization rules
* Some more if conditions in the code
* Change RemoteCollationIdExpression and some pg12/pg13 comments
* Remove some more normalization rules
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
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
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.
1) Rename CONNECTION_PER_PLACEMENT to REQUIRE_CLEAN_CONNECTION. This is
mostly to make things clear as the new name reveals more.
2) We also make sure that mark all the copy connections critical,
even if they are accessed earlier in the transction
* 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
adaptive_executor: sort includes, use foreach_ptr, remove lies from FinishDistributedExecution docs
connection_management: rename msecs, which isn't milliseconds
placement_connection: small typos
Fixes#3331
In #2389, we've implemented support for partitioned tables with rep > 1.
The implementation is limiting the use of modification queries on the
partitions. In fact, we error out when any partition is modified via
EnsurePartitionTableNotReplicated().
However, we seem to forgot an important case, where the parent table's
partition is marked as INVALID. In that case, at least one of the partition
becomes INVALID. However, we do not mark partitions as INVALID ever.
If the user queries the partition table directly, Citus could happily send
the query to INVALID placements -- which are not marked as INVALID.
This PR fixes it by marking the placements of the partitions as INVALID
as well.
The shard placement repair logic already re-creates all the partitions,
so should be fine in that front.
* 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.
/*
* 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.
*/
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 |
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
Before this commit, shardPlacements were identified with shardId, nodeName
and nodeport. Instead of using nodeName and nodePort, we now use nodeId
since it apparently has performance benefits in several places in the
code.
Make sure that intermediate results use a connection that is
not associated with any placement. That is useful in two ways:
- More complex queries can be executed with CTEs
- Safely use the same connections when there is a foreign key
to reference table from a distributed table, which needs to
use the same connection for modifications since the reference
table might cascade to the distributed table.
Postgres provides OS agnosting formatting macros for
formatting 64 bit numbers. Replaced %ld %lu with
INT64_FORMAT and UINT64_FORMAT respectively.
Also found some incorrect usages of formatting
flags and fixed them.
Comes with a few changes:
- Change the signature of some functions to accept groupid
- InsertShardPlacementRow
- DeleteShardPlacementRow
- UpdateShardPlacementState
- NodeHasActiveShardPlacements returns true if the group the node is a
part of has any active shard placements
- TupleToShardPlacement now returns ShardPlacements which have NULL
nodeName and nodePort.
- Populate (nodeName, nodePort) when creating ShardPlacements
- Disallow removing a node if it contains any shard placements
- DeleteAllReferenceTablePlacementsFromNode matches based on group. This
doesn't change behavior for now (while there is only one node per
group), but means in the future callers should be careful about
calling it on a secondary node, it'll delete placements on the primary.
- Create concept of a GroupShardPlacement, which represents an actual
tuple in pg_dist_placement and is distinct from a ShardPlacement,
which has been resolved to a specific node. In the future
ShardPlacement should be renamed to NodeShardPlacement.
- Create some triggers which allow existing code to continue to insert
into and update pg_dist_shard_placement as if it still existed.
- Break CheckShardPlacements into multiple functions (The most important
is MarkFailedShardPlacements), so that we can get rid of the global
CoordinatedTransactionUses2PC.
- Call MarkFailedShardPlacements in the router executor, so we mark
shards as invalid and stop using them while inside transaction blocks.