Clang 13 complains about a suspicious string concatenation. It thinks we
might have missed a comma. This adds parentheses to make it clear that
concatenation is indeed what we meant.
In the past, we allowed users to manually switch to 1PC
(e.g., one phase commit). However, with this commit, we
don't. All multi-shard modifications are done via 2PC.
With Citus 9.0, we introduced `citus.single_shard_commit_protocol` which
defaults to 2PC.
With this commit, we prevent any user to set it to 1PC and drop support
for `citus.single_shard_commit_protocol`.
Although this might add some overhead for users, it is already the default
behaviour (so less likely) and marking placements as INVALID is much
worse.
Since PG14 we can now use binary encoding for arrays and composite types
that contain user defined types. This was fixed in this commit in
Postgres: 670c0a1d47
This change starts using that knowledge, by not necessarily falling back
to text encoding anymore for those types.
While doing this and testing a bit more I found various cases where
binary encoding would fail that our checks didn't cover. This fixes
those cases and adds tests for those. It also fixes EXPLAIN ANALYZE
never using binary encoding, which was a leftover of workaround that
was not necessary anymore.
Finally, it changes the default for both `citus.enable_binary_protocol`
and `citus.binary_worker_copy_format` to `true` for PG14 and up. In our
cloud offering `binary_worker_copy_format` already was true by default.
`enable_binary_protocol` had some bug with MX and user defined types,
this bug was fixed by the above mentioned fixes.
`tcp_user_timeout` is the awesome relatively unknown big brother of the
TCP keepalive related options. Instead of depending on keepalives being
sent, this determines that a socket is dead by waiting at most N seconds
for an ack of data that it has sent. It's exposed in libpq starting from
PG12.
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.
Without this change the rebalancer progress monitor gets the shard sizes
from the `shardlength` column in `pg_dist_placement`. This column needs to
be updated manually by calling `citus_update_table_statistics`.
However, `citus_update_table_statistics` could lead to distributed
deadlocks while database traffic is on-going (see #4752).
To work around this we don't use `shardlength` column anymore. Instead
for every rebalance we now fetch all shard sizes on the fly.
Two additional things this does are:
1. It adds tests for the rebalance progress function.
2. If a shard move cannot be done because a source or target node is
unreachable, then we error in stop the rebalance, instead of showing
a warning and continuing. When using the by_disk_size rebalance
strategy it's not safe to continue with other moves if a specific
move failed. It's possible that the failed move made space for the
next move, and because the failed move never happened this space now
does not exist.
3. Adds two new columns to the result of `get_rebalancer_progress` which
shows the size of the shard on the source and target node.
Fixes#4930
DESCRIPTION: Add support for ALTER DATABASE OWNER
This adds support for changing the database owner. It achieves this by marking the database as a distributed object. By marking the database as a distributed object it will look for its dependencies and order the user creation commands (enterprise only) before the alter of the database owner. This is mostly important when adding new nodes.
By having the database marked as a distributed object it can easily understand for which `ALTER DATABASE ... OWNER TO ...` commands to propagate by resolving the object address of the database and verifying it is a distributed object, and hence should propagate changes of owner ship to all workers.
Given the ownership of the database might have implications on subsequent commands in transactions we force sequential mode for transactions that have a `ALTER DATABASE ... OWNER TO ...` command in them. This will fail the transaction with meaningful help when the transaction already executed parallel statements.
By default the feature is turned off since roles are not automatically propagated, having it turned on would cause hard to understand errors for the user. It can be turned on by the user via setting the `citus.enable_alter_database_owner`.
Comment from the code:
/*
* Iterate until all the tasks are finished. Once all the tasks
* are finished, ensure that that all the connection initializations
* are also finished. Otherwise, those connections are terminated
* abruptly before they are established (or failed). Instead, we let
* the ConnectionStateMachine() to properly handle them.
*
* Note that we could have the connections that are not established
* as a side effect of slow-start algorithm. At the time the algorithm
* decides to establish new connections, the execution might have tasks
* to finish. But, the execution might finish before the new connections
* are established.
*/
Note that the abruptly terminated connections lead to the following errors:
2020-11-16 21:09:09.800 CET [16633] LOG: could not accept SSL connection: Connection reset by peer
2020-11-16 21:09:09.872 CET [16657] LOG: could not accept SSL connection: Undefined error: 0
2020-11-16 21:09:09.894 CET [16667] LOG: could not accept SSL connection: Connection reset by peer
To easily reproduce the issue:
- Create a single node Citus
- Add the coordinator to the metadata
- Create a distributed table with shards on the coordinator
- f.sql: select count(*) from test;
- pgbench -f /tmp/f.sql postgres -T 12 -c 40 -P 1 or pgbench -f /tmp/f.sql postgres -T 12 -c 40 -P 1 -C
With this commit, the executor becomes smarter about refrain to open
new connections. The very basic example is that, if the connection
establishments take 1000ms and task executions as 5 msecs, the executor
becomes smart enough to not establish new connections.
DESCRIPTION: introduce `citus.local_hostname` GUC for connections to the current node
Citus once in a while needs to connect to itself for some systems operations. This used to be hardcoded to `localhost`. The hardcoded hostname causes some issues, for example in environments where `sslmode=verify-full` is required. It is not always desirable or even feasible to get `localhost` as an alt name on the certificate.
By introducing a GUC to use when connecting to the current instance the user has more control what network path is used and what hostname is required to be present in the server certificate.
Every move in the rebalancer algorithm results in an improvement in the
balance. However, even if the improvement in the balance was very small
the move was still chosen. This is especially problematic if the shard
itself is very big and the move will take a long time.
This changes the rebalancer algorithm to take the relative size of the
balance improvement into account when choosing moves. By default a move
will not be chosen if it improves the balance by less than half of the
size of the shard. An extra argument is added to the rebalancer
functions so that the user can decide to lower the default threshold if
the ignored move is wanted anyway.
* When moving a shard to a new node ensure there is enough space
* Add WairForMiliseconds time utility
* Add more tests and increase readability
* Remove the retry loop and use a single udf for disk stats
* Address review
* address review
Co-authored-by: Jelte Fennema <github-tech@jeltef.nl>
As long as the VALUES clause contains constant values, we should not
recursively plan the queries/CTEs.
This is a follow-up work of #1805. So, we can easily apply OUTER join
checks as if VALUES clause is a reference table/immutable function.
When COPY is used for copying into co-located files, it was
not allowed to use local execution. The primary reason was
Citus treating co-located intermediate results as co-located
shards, and COPY into the distributed table was done via
"format result". And, local execution of such COPY commands
was not implemented.
With this change, we implement support for local execution with
"format result". To do that, we use the buffer for every file
on shardState->copyOutState, similar to how local copy on
shards are implemented. In fact, the logic is similar to
local copy on shards, but instead of writing to the shards,
Citus writes the results to a file.
The logic relies on LOCAL_COPY_FLUSH_THRESHOLD, and flushes
only when the size exceeds the threshold. But, unlike local
copy on shards, in this case we write the headers and footers
just once.
When enabled any foreign keys between local tables and reference
tables supported by converting the local table to a citus local
table.
When the coordinator is not in the metadata, the logic is disabled
as foreign keys are not allowed in this configuration.
Instead of sending NULL's over a network, we now convert the subqueries
in the form of:
SELECT t.a, NULL, NULL FROM (SELECT a FROM table)t;
And we recursively plan the inner part so that we don't send the NULL's
over network. We still need the NULLs in the outer subquery because we
currently don't have an easy way of updating all the necessary places in
the query.
Add some documentation for how the conversion is done
Baseinfo also has pushed down filters etc, so it makes more sense to use
BaseRestrictInfo to determine what columns have constant equality
filters.
Also RteIdentity is used for removing conversion candidates instead of
rteIndex.
When doing local-distributed table joins we convert one of them to
subquery. The current policy is that we convert distributed tables to
subquery if it has a unique index on a column that has unique
index(primary key also has a unique index).
The logical planner cannot handle joins between local and distributed table.
Instead, we can recursively plan one side of the join and let the logical
planner handle the rest.
Our algorithm is a little smart, trying not to recursively plan distributed
tables, but favors local tables.
When Citus needs to parallelize queries on the local node (e.g., the node
executing the distributed query and the shards are the same), we need to
be mindful about the connection management. The reason is that the client
backends that are running distributed queries are competing with the client
backends that Citus initiates to parallelize the queries in order to get
a slot on the max_connections.
In that regard, we implemented a "failover" mechanism where if the distributed
queries cannot get a connection, the execution failovers the tasks to the local
execution.
The failover logic is follows:
- As the connection manager if it is OK to get a connection
- If yes, we are good.
- If no, we fail the workerPool and the failure triggers
the failover of the tasks to local execution queue
The decision of getting a connection is follows:
/*
* For local nodes, solely relying on citus.max_shared_pool_size or
* max_connections might not be sufficient. The former gives us
* a preview of the future (e.g., we let the new connections to establish,
* but they are not established yet). The latter gives us the close to
* precise view of the past (e.g., the active number of client backends).
*
* Overall, we want to limit both of the metrics. The former limit typically
* kics in under regular loads, where the load of the database increases in
* a reasonable pace. The latter limit typically kicks in when the database
* is issued lots of concurrent sessions at the same time, such as benchmarks.
*/
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.
In postmasters execution of _PG_init, IsUnderPostmaster will be false and
we want to do the cleanup at that time only, otherwise there is a chance that
there will be parallel queries and we might do a cleanup for things that are
already in use.
Add sort method parameter for regression tests
Fix check-style
Change sorting method parameters to enum
Polish
Add task fields to OutTask
Add test into multi_explain
Fix isolation test
* Hide citus.subquery_pushdown flag
This flag is dangerous and could likely to let queries
return wrong results.
The flag has a very specific purpose for a very specific
data distribution and query structure. In those cases, when
the flag is set, the user can skip recursive planning altogether
*at their own risk*.
The meaning of the flag is that "I know what I'm doing such that
the query structure/data distribution is on my control, so Citus
can skip many correctness checks".
For regular users, enabling this flag is discouraged. We have to
keep the support only for backward compatibility for some users.
In addition to that, give a NOTICE to discourage new users to
use it.
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.
* use adaptive executor even if task-tracker is set
* Update check-multi-mx tests for adaptive executor
Basically repartition joins are enabled where necessary. For parallel
tests max adaptive executor pool size is decresed to 2, otherwise we
would get too many clients error.
* Update limit_intermediate_size test
It seems that when we use adaptive executor instead of task tracker, we
exceed the intermediate result size less in the test. Therefore updated
the tests accordingly.
* Update multi_router_planner
It seems that there is one problem with multi_router_planner when we use
adaptive executor, we should fix the following error:
+ERROR: relation "authors_range_840010" does not exist
+CONTEXT: while executing command on localhost:57637
* update repartition join tests for check-multi
* update isolation tests for repartitioning
* Error out if shard_replication_factor > 1 with repartitioning
As we are removing the task tracker, we cannot switch to it if
shard_replication_factor > 1. In that case, we simply error out.
* Remove MULTI_EXECUTOR_TASK_TRACKER
* Remove multi_task_tracker_executor
Some utility methods are moved to task_execution_utils.c.
* Remove task tracker protocol methods
* Remove task_tracker.c methods
* remove unused methods from multi_server_executor
* fix style
* remove task tracker specific tests from worker_schedule
* comment out task tracker udf calls in tests
We were using task tracker udfs to test permissions in
multi_multiuser.sql. We should find some other way to test them, then we
should remove the commented out task tracker calls.
* remove task tracker test from follower schedule
* remove task tracker tests from multi mx schedule
* Remove task-tracker specific functions from worker functions
* remove multi task tracker extra schedule
* Remove unused methods from multi physical planner
* remove task_executor_type related things in tests
* remove LoadTuplesIntoTupleStore
* Do initial cleanup for repartition leftovers
During startup, task tracker would call TrackerCleanupJobDirectories and
TrackerCleanupJobSchemas to clean up leftover directories and job
schemas. With adaptive executor, while doing repartitions it is possible
to leak these things as well. We don't retry cleanups, so it is possible
to have leftover in case of errors.
TrackerCleanupJobDirectories is renamed as
RepartitionCleanupJobDirectories since it is repartition specific now,
however TrackerCleanupJobSchemas cannot be used currently because it is
task tracker specific. The thing is that this function is a no-op
currently.
We should add cleaning up intermediate schemas to DoInitialCleanup
method when that problem is solved(We might want to solve it in this PR
as well)
* Revert "remove task tracker tests from multi mx schedule"
This reverts commit 03ecc0a681.
* update multi mx repartition parallel tests
* not error with task_tracker_conninfo_cache_invalidate
* not run 4 repartition queries in parallel
It seems that when we run 4 repartition queries in parallel we get too
many clients error on CI even though we don't get it locally. Our guess
is that, it is because we open/close many connections without doing some
work and postgres has some delay to close the connections. Hence even
though connections are removed from the pg_stat_activity, they might
still not be closed. If the above assumption is correct, it is unlikely
for it to happen in practice because:
- There is some network latency in clusters, so this leaves some times
for connections to be able to close
- Repartition joins return some data and that also leaves some time for
connections to be fully closed.
As we don't get this error in our local, we currently assume that it is
not a bug. Ideally this wouldn't happen when we get rid of the
task-tracker repartition methods because they don't do any pruning and
might be opening more connections than necessary.
If this still gives us "too many clients" error, we can try to increase
the max_connections in our test suite(which is 100 by default).
Also there are different places where this error is given in postgres,
but adding some backtrace it seems that we get this from
ProcessStartupPacket. The backtraces can be found in this link:
https://circleci.com/gh/citusdata/citus/138702
* Set distributePlan->relationIdList when it is needed
It seems that we were setting the distributedPlan->relationIdList after
JobExecutorType is called, which would choose task-tracker if
replication factor > 1 and there is a repartition query. However, it
uses relationIdList to decide if the query has a repartition query, and
since it was not set yet, it would always think it is not a repartition
query and would choose adaptive executor when it should choose
task-tracker.
* use adaptive executor even with shard_replication_factor > 1
It seems that we were already using adaptive executor when
replication_factor > 1. So this commit removes the check.
* remove multi_resowner.c and deprecate some settings
* remove TaskExecution related leftovers
* change deprecated API error message
* not recursively plan single relatition repartition subquery
* recursively plan single relation repartition subquery
* test depreceated task tracker functions
* fix overlapping shard intervals in range-distributed test
* fix error message for citus_metadata_container
* drop task-tracker deprecated functions
* put the implemantation back to worker_cleanup_job_schema_cachesince citus cloud uses it
* drop some functions, add downgrade script
Some deprecated functions are dropped.
Downgrade script is added.
Some gucs are deprecated.
A new guc for repartition joins bucket size is added.
* order by a test to fix flappiness
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.
We wrap worker tasks in worker_save_query_explain_analyze() so we can fetch
their explain output later by a call worker_last_saved_explain_analyze().
Fixes#3519Fixes#2347Fixes#2613Fixes#621
The previous default was 5 seconds, and we change it to 30 seconds.
The main motivation for this is that for busy clusters, 5 seconds
can be too aggressive. Especially with connection throttling, the servers
might be kept busy for a really long time, and users may see the
connection errors more frequently.
We've done some sanity checks, for really quick queries (like
`SELECT count(*) from table`), 30 seconds is a decent value even
if users execute 300 distributed queries on the coordinator. We've
verified this on Hyperscale(Citus).
DESCRIPTION: Alter role only works for citus managed roles
Alter role was implemented before we implemented good role management that hooks into the object propagation framework. This is a refactor of all alter role commands that have been implemented to
- be on by default
- only work for supported roles
- make the citus extension owner a supported role
Instead of distributing the alter role commands for roles at the beginning of the node activation role it now _only_ executes the alter role commands for all users in all databases and in the current database.
In preparation of full role support small refactors have been done in the deparser.
Earlier tests targeting other roles than the citus extension owner have been either slightly changed or removed to be put back where we have full role support.
Fixes#2549
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.
We cache connections between nodes in our connection management code.
This is good for speed. For security this can be a problem though. If
the user changes settings related to TLS encryption they want those to
be applied to future queries. This is especially important when they did
not have TLS enabled before and now they want to enable it. This can
normally be achieved by changing citus.node_conninfo. However, because
connections are not reopened there will still be old connections that
might not be encrypted at all.
This commit changes that by marking all connections to be shutdown at
the end of their current transaction. This way running transactions will
succeed, even if placement requires connections to be reused for this
transaction. But after this transaction completes any future statements
will use a connection created with the new connection options.
If a connection is requested and a connection is found that is marked
for shutdown, then we don't return this connection. Instead a new one is
created. This is needed to make sure that if there are no running
transactions, then the next statement will not use an old cached
connection, since connections are only actually shutdown at the end of a
transaction.
Citus coordinator (or MX nodes) caches `citus.max_cached_conns_per_worker` connections
per node. This means that, those connections are not terminated after each statement.
Instead, cached to avoid the cost of re-establishment. This is crucial for OLTP performance.
The problem with that approach is that, we never properly handle the termnation of
those cached connections. For instance, when a session on the coordinator disconnects,
you'd see the following logs on the workers:
```
2020-03-20 09:13:39.454 CET [64028] LOG: could not receive data from client: Connection reset by peer
```
With this patch, we're terminating the cached connections properly at the end of the connection.
DESCRIPTION: Replace the query planner for the coordinator part with the postgres planner
Closes#2761
Citus had a simple rule based planner for the query executed on the query coordinator. This planner grew over time with the addigion of SQL support till it was getting close to the functionality of the postgres planner. Except the code was brittle and its complexity rose which made it hard to add new SQL support.
Given its resemblance with the postgres planner it was a long outstanding wish to replace our hand crafted planner with the well supported postgres planner. This patch replaces our planner with a call to postgres' planner.
Due to the functionality of the postgres planner we needed to support both projections and filters/quals on the citus custom scan node. When a sort operation is planned above the custom scan it might require fields to be reordered in the custom scan before returning the tuple (projection). The postgres planner assumes every custom scan node implements projections. Because we controlled the plan that was created we prevented reordering in the custom scan and never had implemented it before.
A same optimisation applies to having clauses that could have been where clauses. Instead of applying the filter as a having on the aggregate it will push it down into the plan which could reach a custom scan node.
For both filters and projections we have implemented them when tuples are read from the tuple store. If no projections or filters are required it will directly return the tuple from the tuple store. Otherwise it will loop tuples from the tuple store through the filter and projection until a tuple is found and returned.
Besides filters being pushed down a side effect of having quals that could have been a where clause is that a call to read intermediate result could be called before the first tuple is fetched from the custom scan. This failed because the intermediate result would only be pulled to the coordinator on the first tuple fetch. To overcome this problem we do run the distributed subplans now before we run the postgres executor. This ensures the intermediate result is present on the coordinator in time. We do account for total time instrumentation by removing the instrumentation before handing control to the psotgres executor and update the timings our self.
For future SQL support it is enough to create a valid query structure for the part of the query to be executed on the query coordinating node. As a utility we do serialise and print the query at debug level4 for engineers to inspect what kind of query is being planned on the query coordinator.
As that is powerful and cause metadata inconsistency. See the following steps:
(Note that we cannot use PGC_SUSET because on Citus MX we need this flag for non-
superusers as well)
```SQL
CREATE TABLE test_ref_table(key int);
SELECT create_reference_table('test_ref_table');
SELECT logicalrelid, logicalrelid::oid FROM pg_dist_partition;
┌────────────────┬──────────────┐
│ logicalrelid │ logicalrelid │
├────────────────┼──────────────┤
│ test_ref_table │ 16831 │
└────────────────┴──────────────┘
(1 row)
Time: 0.929 ms
SELECT relname FROM pg_class WHERE oid = 16831;
┌────────────────┐
│ relname │
├────────────────┤
│ test_ref_table │
└────────────────┘
(1 row)
Time: 0.785 ms
SET citus.enable_ddl_propagation TO off;
DROP TABLE test_ref_table ;
SELECT logicalrelid, logicalrelid::oid FROM pg_dist_partition;
┌──────────────┬──────────────┐
│ logicalrelid │ logicalrelid │
├──────────────┼──────────────┤
│ 16831 │ 16831 │
└──────────────┴──────────────┘
(1 row)
Time: 0.972 ms
SELECT relname FROM pg_class WHERE oid = 16831;
┌─────────┐
│ relname │
├─────────┤
└─────────┘
(0 rows)
Time: 0.908 ms
SELECT master_add_node('localhost', 9703);
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.
Time: 5.028 ms
!>
```
In this commit, we're introducing a way to prevent CTE inlining via a GUC.
The GUC is used in all the tests where PG 11 and PG 12 tests would diverge
otherwise.
Note that, in PG 12, the restriction information for CTEs are generated. It
means that for some queries involving CTEs, Citus planner (router planner/
pushdown planner) may behave differently. So, via the GUC, we prevent
tests to diverge on PG 11 vs PG 12.
When we drop PG 11 support, we should get rid of the GUC, and mark
relevant ctes as MATERIALIZED, which does the same thing.
* 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
In plain words, each distributed plan pulls the necessary intermediate
results to the worker nodes that the plan hits. This is primarily useful
in three ways.
(i) If the distributed plan that uses intermediate
result(s) is a router query, then the intermediate results are only
broadcasted to a single node.
(ii) If a distributed plan consists of only intermediate results, which
is not uncommon, the intermediate results are broadcasted to a single
node only.
(iii) If a distributed query hits a sub-set of the shards in multiple
workers, the intermediate results will be broadcasted to the relevant
node(s).
The final item (iii) becomes crucial for append/range distributed
tables where typically the distributed queries hit a small subset of
shards/workers.
To do this, for each query that Citus creates a distributed plan, we keep
track of the subPlans used in the queryTree, and save it in the distributed
plan. Just before Citus executes each subPlan, Citus first keeps track of
every worker node that the distributed plan hits, and marks every subPlan
should be broadcasted to these nodes. Later, for each subPlan which is a
distributed plan, Citus does this operation recursively since these
distributed plans may access to different subPlans, and those have to be
recorded as well.
* 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.
See #3125 for details on each item.
* Remove real-time/router executor tests-1
These are the ones which doesn't have '_%d' in the test
output files.
* Remove real-time/router executor tests-2
These are the ones which has in the test
output files.
* Move the tests outputs to correct place
* Make sure that single shard commits use 2PC on adaptive executor
It looks like we've messed the tests in #2891. Fixing back.
* Use adaptive executor for all router queries
This becomes important because when task-tracker is picked, we
used to pick router executor, which doesn't make sense.
* Remove explicit references to real-time/router executors in the tests
* JobExecutorType never picks real-time/router executors
* Make sure to go incremental in test output numbers
* Even users cannot pick real-time anymore
* Do not use real-time/router custom scans
* Get rid of unnecessary normalizations
* Reflect unneeded normalizations
* Get rid of unnecessary test output file
DESCRIPTION: Provide a GUC to turn of the new dependency propagation functionality
In the case the dependency propagation functionality introduced in 9.0 causes issues to a cluster of a user they can turn it off almost completely. The only dependency that will still be propagated and kept track of is the schema to emulate the old behaviour.
GUC to change is `citus.enable_object_propagation`. When set to `false` the functionality will be mostly turned off. Be aware that objects marked as distributed in `pg_dist_object` will still be kept in the catalog as a distributed object. Alter statements to these objects will not be propagated to workers and may cause desynchronisation.
DESCRIPTION: Add feature flag to turn off create type propagation
When `citus.enable_create_type_propagation` is set to `false` citus will not propagate `CREATE TYPE` statements to the workers. Types are still distributed when tables that depend on these types are distributed.
/*
* 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.
*/
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
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.
The feature is only intended for getting consistent outputs for the regression tests.
RETURNING does not have any ordering gurantees and with unified executor, the ordering
of query executions on the shards are also becoming unpredictable. Thus, we're enforcing
ordering when a GUC is set.
We implicitly add an `ORDER BY` something equivalent of
`
RETURNING expr1, expr2, .. ,exprN
ORDER BY expr1, expr2, .. ,exprN
`
As described in the code comments as well, this is probably not the most
performant approach we could implement. However, since we're only
targeting regression tests, I don't see any issues with that. If we
decide to expand this to a feature to users, we should revisit the
implementation and improve the performance.
This commit has two goals:
(a) Ensure to access both edges of the allocated stack
(b) Ensure that any compiler optimizations to prevent the
function optimized away.
Stack size after the patch:
sudo grep -A 1 stack /proc/2119/smaps
7ffe305a6000-7ffe307a9000 rw-p 00000000 00:00 0 [stack]
Size: 2060 kB
Stack size before the patch:
sudo grep -A 1 stack /proc/3610/smaps
7fff09957000-7fff09978000 rw-p 00000000 00:00 0 [stack]
Size: 132 kB
In this context, we define "Fast Path Planning for SELECT" as trivial
queries where Citus can skip relying on the standard_planner() and
handle all the planning.
For router planner, standard_planner() is mostly important to generate
the necessary restriction information. Later, the restriction information
generated by the standard_planner is used to decide whether all the shards
that a distributed query touches reside on a single worker node. However,
standard_planner() does a lot of extra things such as cost estimation and
execution path generations which are completely unnecessary in the context
of distributed planning.
There are certain types of queries where Citus could skip relying on
standard_planner() to generate the restriction information. For queries
in the following format, Citus does not need any information that the
standard_planner() generates:
SELECT ... FROM single_table WHERE distribution_key = X; or
DELETE FROM single_table WHERE distribution_key = X; or
UPDATE single_table SET value_1 = value_2 + 1 WHERE distribution_key = X;
Note that the queries might not be as simple as the above such that
GROUP BY, WINDOW FUNCIONS, ORDER BY or HAVING etc. are all acceptable. The
only rule is that the query is on a single distributed (or reference) table
and there is a "distribution_key = X;" in the WHERE clause. With that, we
could use to decide the shard that a distributed query touches reside on
a worker node.
The file handling the utility functions (DDL) for citus organically grew over time and became unreasonably large. This refactor takes that file and refactored the functionality into separate files per command. Initially modeled after the directory and file layout that can be found in postgres.
Although the size of the change is quite big there are barely any code changes. Only one two functions have been added for readability purposes:
- PostProcessIndexStmt which is extracted from PostProcessUtility
- PostProcessAlterTableStmt which is extracted from multi_ProcessUtility
A README.md has been added to `src/backend/distributed/commands` describing the contents of the module and every file in the module.
We need more documentation around the overloading of the COPY command, for now the boilerplate has been added for people with better knowledge to fill out.
Each PostgreSQL backend starts with a predefined amount of stack and this stack
size can be increased if there is a need. However, stack size increase during
high memory load may cause unexpected crashes, because if there is not enough
memory for stack size increase, there is nothing to do for process apart from
crashing. An interesting thing is; the process would get OOM error instead of
crash, if the process had an explicit memory request (with palloc) for example.
However, in the case of stack size increase, there is no system call to get OOM
error, so the process simply crashes.
With this change, we are increasing the stack size explicitly by requesting extra
memory from the stack, so that, even if there is not memory, we can at least get
an OOM instead of a crash.
This commit by default enables hiding shard names on MX workers
by simple replacing `pg_table_is_visible()` calls with
`citus_table_is_visible()` calls on the MX worker nodes. The latter
function filters out tables that are known to be shards.
The main motivation of this change is a better UX. The functionality
can be opted out via a GUC.
We also added two views, namely citus_shards_on_worker and
citus_shard_indexes_on_worker such that users can query
them to see the shards and their corresponding indexes.
We also added debug messages such that the filtered tables can
be interactively seen by setting the level to DEBUG1.
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.
To support more flexible (i.e. not at compile-time) specification of
libpq connection parameters, this change adds a new GUC, node_conninfo,
which must be a space-separated string of key-value pairs suitable for
parsing by libpq's connection establishment methods.
To avoid rebuilding and parsing these values at connection time, this
change also adds a cache in front of the configuration params to permit
immediate use of any previously-calculated parameters.
* Change worker_hash_partition_table() such that the
divergence between Citus planner's hashing and
worker_hash_partition_table() becomes the same.
* Rename single partitioning to single range partitioning.
* Add single hash repartitioning. Basically, logical planner
treats single hash and range partitioning almost equally.
Physical planner, on the other hand, treats single hash and
dual hash repartitioning almost equally (except for JoinPruning).
* Add a new GUC to enable this feature
After this change all the logic related to shard data fetch logic
will be removed. Planner won't plan any ShardFetchTask anymore.
Shard fetch related steps in real time executor and task-tracker
executor have been removed.
This commit introduces a new GUC to limit the intermediate
result size which we handle when we use read_intermediate_result
function for CTEs and complex subqueries.
Adds ```citus.enable_statistics_collection``` GUC variable, which ```true``` by default, unless built without libcurl. If statistics collection is enabled, sends basic usage data to Citus servers every 24 hours.
The data that is collected consists of:
- Citus version
- OS name & release
- Hardware Id
- Number of tables, rounded to next power of 2
- Size of data, rounded to next power of 2
- Number of workers
We added a new GUC citus.log_distributed_deadlock_detection
which is off by default. When set to on, we log some debug messages
related to the distributed deadlock to the server logs.
With this commit, the maintenance deamon starts to check for
distributed deadlocks.
We also introduced a GUC variable (distributed_deadlock_detection_factor)
whose value is multiplied with Postgres' deadlock_timeout. Setting
it to -1 disables the distributed deadlock detection.
We send SIGINT to a backend that is cancelled due to a deadlock. That
approach ends up being a very confusing error message.
With this commit we intercept the error messages and show a more
meaningful error message to the user.
Now that we already have the necessary infrastructure for detecting
distributed deadlocks. Thus, we don't need enable_deadlock_prevention
which is purely intended for preventing some forms of distributed
deadlocks.
This GUC has two settings, 'always' and 'never'. When it's set to
'never' all behavior stays exactly as it was prior to this commit. When
it's set to 'always' only SELECT queries are allowed to run, and only
secondary nodes are used when processing those queries.
Add some helper functions:
- WorkerNodeIsSecondary(), checks the noderole of the worker node
- WorkerNodeIsReadable(), returns whether we're currently allowed to
read from this node
- ActiveReadableNodeList(), some functions (namely, the ones on the
SELECT path) don't require working with Primary Nodes. They should call
this function instead of ActivePrimaryNodeList(), because the latter
will error out in contexts where we're not allowed to write to nodes.
- ActiveReadableNodeCount(), like the above, replaces
ActivePrimaryNodeCount().
- EnsureModificationsCanRun(), error out if we're not currently allowed
to run queries which modify data. (Either we're in read-only mode or
use_secondary_nodes is set)
Some parts of the code were switched over to use readable nodes instead
of primary nodes:
- Deadlock detection
- DistributedTableSize,
- the router, real-time, and task tracker executors
- ShardPlacement resolution
maxTaskStringSize determines the size of worker query string.
It was originally hard coded to a specific value. This has caused
issues at some users. Since it determines initial shared memory
allocation, we did not want to set it to an arbitrary higher number.
Instead made it configurable.
This commit introduces a new GUC variable max_task_string_size
Changes in this variable requires restart to be in effect.
This commit adds distributed transaction id infrastructure in
the scope of distributed deadlock detection.
In general, the distributed transaction id consists of a tuple
in the form of: `(databaseId, initiatorNodeIdentifier, transactionId,
timestamp)`.
Briefly, we add a shared memory block on each node, which holds some
information per backend (i.e., an array `BackendData backends[MaxBackends]`).
Later, on each coordinated transaction, Citus sends
`SELECT assign_distributed_transaction_id()` right after `BEGIN`.
For that backend on the worker, the distributed transaction id is set to
the values assigned via the function call.
The aim of the above is to correlate the transactions on the coordinator
to the transactions on the worker nodes.
Adds support for PostgreSQL 10 by copying in the requisite ruleutils
and updating all API usages to conform with changes in PostgreSQL 10.
Most changes are fairly minor but they are numerous. One particular
obstacle was the change in \d behavior in PostgreSQL 10's psql; I had
to add SQL implementations (views, mostly) to mimic the pre-10 output.
In this PR, we aim to deduce whether each of the RTE_RELATION
is joined with at least on another RTE_RELATION on their partition keys. If each
RTE_RELATION follows the above rule, we can conclude that all RTE_RELATIONs are
joined on their partition keys.
In order to do that, we invented a new equivalence class namely:
AttributeEquivalenceClass. In very simple words, a AttributeEquivalenceClass is
identified by an unique id and consists of a list of AttributeEquivalenceMembers.
Each AttributeEquivalenceMember is designed to identify attributes uniquely within the
whole query. The necessity of this arise since varno attributes are defined within
a single level of a query. Instead, here we want to identify each RTE_RELATION uniquely
and try to find equality among each RTE_RELATION's partition key.
Whenever we find an equality clause A = B, where both A and B originates from
relation attributes (i.e., not random expressions), we create an
AttributeEquivalenceClass to record this knowledge. If we later find another
equivalence B = C, we create another AttributeEquivalenceClass. Finally, we can
apply transitity rules and generate a new AttributeEquivalenceClass which includes
A, B and C.
Note that equality among the members are identified by the varattno and rteIdentity.
Each equality among RTE_RELATION is saved using an AttributeEquivalenceClass where
each member attribute is identified by a AttributeEquivalenceMember. In the final
step, we try generate a common attribute equivalence class that holds as much as
AttributeEquivalenceMembers whose attributes are a partition keys.
With this change, we start to error out if loaded citus binaries does not match
the available major version or installed citus extension version. In this case
we force user to restart the server or run ALTER EXTENSION depending on the
situation
Custom Scan is a node in the planned statement which helps external providers
to abstract data scan not just for foreign data wrappers but also for regular
relations so you can benefit your version of caching or hardware optimizations.
This sounds like only an abstraction on the data scan layer, but we can use it
as an abstraction for our distributed queries. The only thing we need to do is
to find distributable parts of the query, plan for them and replace them with
a Citus Custom Scan. Then, whenever PostgreSQL hits this custom scan node in
its Vulcano style execution, it will call our callback functions which run
distributed plan and provides tuples to the upper node as it scans a regular
relation. This means fewer code changes, fewer bugs and more supported features
for us!
First, in the distributed query planner phase, we create a Custom Scan which
wraps the distributed plan. For real-time and task-tracker executors, we add
this custom plan under the master query plan. For router executor, we directly
pass the custom plan because there is not any master query. Then, we simply let
the PostgreSQL executor run this plan. When it hits the custom scan node, we
call the related executor parts for distributed plan, fill the tuple store in
the custom scan and return results to PostgreSQL executor in Vulcano style,
a tuple per XXX_ExecScan() call.
* Modify planner to utilize Custom Scan node.
* Create different scan methods for different executors.
* Use native PostgreSQL Explain for master part of queries.