Multi-row & router INSERT's were crashing with local execution if at
least one of the DEFAULT columns were not specified in VALUES list.
This was because, the changes we make on query->values_lists and
query->targetList was sufficient for deparsing given INSERT for remote
execution but not sufficient for local execution.
With this commit, DEFAULT value normalization for multi-row & router
INSERT's is fixed by adding dummy column references for unspecified
DEFAULT columns.
This commit brings following features:
Foreign key support from citus local tables to reference tables
* Foreign key support from reference tables to citus local tables
(only with RESTRICT & NO ACTION behavior)
* ALTER TABLE ENABLE/DISABLE trigger command support
* CREATE/DROP/ALTER trigger command support
and disallows:
* ALTER TABLE ATTACH/DETACH PARTITION commands
* CREATE TABLE <postgres table> ATTACH PARTITION <citus local table>
commands
* Foreign keys from postgres tables to citus local tables
(the other way was already disallowed)
for citus local tables.
Postgres 13 added a new VACUUM option, PARALLEL. It is now supported
in our code as well.
Relevant changelog message on postgres:
Allow VACUUM to process indexes in parallel (Masahiko Sawada, Amit Kapila)
DESCRIPTION: Force aliases in deparsing for queries with anonymous column references
Fixes: #3985
The root cause has todo with discrepancies in the query tree we create. I think in the future we should spend some time on categorising all changes we made to ruleutils and see if we can change the data structure `query` we pass to the deparser to have an actual valid postgres query for the deparser to render.
For now the fix is to keep track, besides changing the names of the entries in the target list, also if we have a reference to an anonymous columns. If there are anonymous columns we set the `printaliases` flag to true which forces the deparser to add the aliases.
Static analysis found an issue where we could dereference `NULL`, because
`CreateDummyPlacement` could return `NULL` when there were no workers. This
PR changes it so that it never returns `NULL`, which was intended by
@marcocitus when doing this change: https://github.com/citusdata/citus/pull/3887/files#r438136433
While adding tests for citus on a single node I also added some more basic
tests and it turns out we error out on repartition joins. This has been
present since `shouldhaveshards` was introduced and is not trivial to fix.
So I created a separate issue for this: https://github.com/citusdata/citus/issues/3996
I recently forgot to add tests to a schedule in two of my PRs. One of
these was caught by review, but the other one was not. This adds a
script to causes CI to ensure that each test in the repo is included in
at least one schedule.
Three tests were found that were currently not part of a schedule. This PR
adds those three tests to a schedule as well and it also fixes some small
issues with these tests.
It was possible to get an assertion error, if a DML command was
cancelled that opened a connection and then "ROLLBACK TO SAVEPOINT" was
used to continue the transaction. The reason for this was that canceling
the transaction might leave the `claimedExclusively` flag on for (some
of) it's connections.
This caused an assertion failure because `CanUseExistingConnection`
would return false and a new connection would be opened, and then there
would be two connections doing DML for the same placement. Which is
disallowed. That this situation caused an assertion failure instead of
an error, means that without asserts this could possibly result in some
visibility bugs, similar to the ones described
https://github.com/citusdata/citus/issues/3867
In #3901 the "Data received from worker(s)" sections were added to EXPLAIN
ANALYZE. After merging @pykello posted some review comments. This addresses
those comments as well as fixing a other issues that I found while addressing
them. The things this does:
1. Fix `EXPLAIN ANALYZE EXECUTE p1` to not increase received data on every
execution
2. Fix `EXPLAIN ANALYZE EXECUTE p1(1)` to not return 0 bytes as received data
allways.
3. Move `EXPLAIN ANALYZE` specific logic to `multi_explain.c` from
`adaptive_executor.c`
4. Change naming of new explain sections to `Tuple data received from node(s)`.
Firstly because a task can reference the coordinator too, so "worker(s)" was
incorrect. Secondly to indicate that this is tuple data and not all network
traffic that was performed.
5. Rename `totalReceivedData` in our codebase to `totalReceivedTupleData` to
make it clearer that it's a tuple data counter, not all network traffic.
6. Actually add `binary_protocol` test to `multi_schedule` (woops)
7. Fix a randomly failing test in `local_shard_execution.sql`.
* Insert select with master query
* Use relid to set custom_scan_tlist varno
* Reviews
* Fixes null check
Co-authored-by: Marco Slot <marco.slot@gmail.com>
DESCRIPTION: Adds support to partially push down tdigest aggregates
tdigest extensions: https://github.com/tvondra/tdigest
This PR implements the partial pushdown of tdigest calculations when possible. The extension adds a tdigest type which can be combined into the same structure. There are several aggregate functions that can be used to get;
- a quantile
- a list of quantiles
- the quantile of a hypothetical value
- a list of quantiles for a list of hypothetical values
These function can work both on values or tdigest types.
Since we can create tdigest values either by combining them, or based on a group of values we can rewrite the aggregates in such a way that most of the computation gets delegated to the compute on the shards. This both speeds up the percentile calculations because the values don't have to be sorted while at the same time making the transfer size from the shards to the coordinator significantly less.
DESCRIPTION: Ignore pruned target list entries in coordinator plan
The postgres planner has the ability to prune target list entries that are proven not used in the output relation. When this happens at the `CitusCustomScan` boundary we need to _not_ return these pruned columns to not upset the rest of the planner.
By using the target list the planner asks us to return we fix issues that lead to Assertion failures, and potentially could be runtime errors when they hit in a production build.
Fixes#3809
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.
We're getting a lot of random failures on CI regarding connection errors. This
works around that by not running that create lots of connections in parallel.
* reimplement ExecuteUtilityTaskListWithoutResults for local utility command execution
* introduce new functions for local execution of utility commands
* change ErrorIfTransactionAccessedPlacementsLocally logic for local utility command execution
* enable local execution for TRUNCATE command on distributed & reference tables
* update existing tests for local utility command execution
* enable local execution for DDL commands on distributed & reference tables
* enable local execution for DROP command on distributed & reference tables
* add normalization rules for cascaded commands
* add new tests for local utility command execution
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.
The root of the problem is that, standard_planner() converts the following qual
```
{OPEXPR
:opno 98
:opfuncid 67
:opresulttype 16
:opretset false
:opcollid 0
:inputcollid 100
:args (
{VAR
:varno 1
:varattno 1
:vartype 25
:vartypmod -1
:varcollid 100
:varlevelsup 0
:varnoold 1
:varoattno 1
:location 45
}
{CONST
:consttype 25
:consttypmod -1
:constcollid 100
:constlen -1
:constbyval false
:constisnull true
:location 51
:constvalue <>
}
)
:location 49
}
```
To
```
(
{CONST
:consttype 16
:consttypmod -1
:constcollid 0
:constlen 1
:constbyval true
:constisnull true
:location -1
:constvalue <>
}
)
```
So, Citus doesn't deal with NULL values in real-time or non-fast path router queries.
And, in the FastPathRouter planner, we check constisnull in DistKeyInSimpleOpExpression().
However, in deferred pruning case, we do not check for isnull for const.
Thus, the fix consists of two parts:
- Let PruneShards() not crash when NULL parameter is passed
- For deferred shard pruning in fast-path queries, explicitly check that we have CONST which is not NULL
With this commit, we're adding the specific tests for CTE inlining.
The test has a different output file for pg 11, because as mentioned
in the previous commits, PG 12 generates more restriction information
for CTEs.
* 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
Test ALTER ROLE doesn't deadlock when coordinator added, or propagate from mx workers
Consolidate wait_until_metadata_sync & verify_metadata to multi_test_helpers
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.
DESCRIPTION: Expression in reference join
Fixed: #2582
This patch allows arbitrary expressions in the join clause when joining to a reference table. An example of such joins could be found in CHbenCHmark queries 7, 8, 9 and 11; `mod((s_w_id * s_i_id),10000) = su_suppkey` and `ascii(substr(c_state,1,1)) = n2.n_nationkey`. Since the join is on a reference table these queries are able to be pushed down to the workers.
To implement these queries we will widen the `IsJoinClause` predicate to not check if the expressions are a type `Var` after stripping the implicit coerciens. Instead we define a join clause when the `Var`'s in a clause come from more than 1 table.
This allows more clauses to pass into the logical planner's `MultiNodeTree(...)` planning function. To compensate for this we tighten down the `LocalJoin`, `SinglePartitionJoin` and `DualPartitionJoin` to check for direct column references when planning. This allows the planner to work with arbitrary join expressions on reference tables.
Phase 1 seeks to implement minimal infrastructure, so does not include:
- dynamic generation of support aggregates to handle multiple arguments
- configuration methods to direct aggregation strategy,
or mark an aggregate's serialize/deserialize as safe to operate across nodes
Aggregates can be distributed when:
- they have a single argument
- they have a combinefunc
- their transition type is not a pseudotype
This is necassery to support Q20 of the CHbenCHmark: #2582.
To summarize the fix: The subquery is converted into an INNER JOIN on a
table. This fixes the issue, since an INNER JOIN on a table is already
supported by the repartion planner.
The way this replacement is happening.:
1. Postgres replaces `col in (subquery)` with a SEMI JOIN (subquery) on col = subquery_result
2. If this subquery is simple enough Postgres will replace it with a
regular read from a table
3. If the subquery returns unique results (e.g. a primary key) Postgres
will convert the SEMI JOIN into an INNER JOIN during the planning. It
will not change this in the rewritten query though.
4. We check if Postgres sends us any SEMI JOINs during its join order
planning, if it doesn't we replace all SEMI JOINs in the rewritten
query with INNER JOIN (which we already support).
Areas for further optimization:
- Don't save subquery results to a local file on the coordinator when the subquery is not in the having clause
- Push the the HAVING with subquery to the workers if there's a group by on the distribution column
- Don't push down the results to the workers when we don't push down the HAVING clause, only the coordinator needs it
Fixes#520Fixes#756Closes#2047
DESCRIPTION: Fix order for enum values and correctly support pg12
PG 12 introduces `ALTER TYPE ... ADD VALUE ...` during transactions. Earlier versions would error out when called in a transaction, hence we connect to workers outside of the transaction which could cause inconsistencies on pg12 now that postgres doesn't error with this syntax anymore.
During the implementation of this fix it became apparent there was an error with the ordering of enum labels when the type was recreated. A patch and test have been included.
DESCRIPTION: Propagate ALTER FUNCTION statements for distributed functions
Using the implemented deparser for function statements to propagate changes to both functions and procedures that are previously distributed.
This PR aims to add all the necessary logic to qualify and deparse all possible `{ALTER|DROP} .. {FUNCTION|PROCEDURE}` queries.
As Procedures are introduced in PG11, the code contains many PG version checks. I tried my best to make it easy to clean up once we drop PG10 support.
Here are some caveats:
- I assumed that the parse tree is a valid one. There are some queries that are not allowed, but still are parsed successfully by postgres planner. Such queries will result in errors in execution time. (e.g. `ALTER PROCEDURE p STRICT` -> `STRICT` action is valid for functions but not procedures. Postgres decides to parse them nevertheless.)
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: Rename remote types during type propagation
To prevent data to be destructed when a remote type differs from the type on the coordinator during type propagation we wanted to rename the type instead of `DROP CASCADE`.
This patch removes the `DROP` logic and adds the creation of a rename statement to a free name.
This PR aims to add the minimal set of changes required to start
distributing functions. You can use create_distributed_function(regproc)
UDF to distribute a function.
SELECT create_distributed_function('add(int,int)');
The function definition should include the param types to properly
identify the correct function that we wish to distribute
DESCRIPTION: Distribute Types to worker nodes
When to propagate
==============
There are two logical moments that types could be distributed to the worker nodes
- When they get used ( just in time distribution )
- When they get created ( proactive distribution )
The just in time distribution follows the model used by how schema's get created right before we are going to create a table in that schema, for types this would be when the table uses a type as its column.
The proactive distribution is suitable for situations where it is benificial to have the type on the worker nodes directly. They can later on be used in queries where an intermediate result gets created with a cast to this type.
Just in time creation is always the last resort, you cannot create a distributed table before the type gets created. A good example use case is; you have an existing postgres server that needs to scale out. By adding the citus extension, add some nodes to the cluster, and distribute the table. The type got created before citus existed. There was no moment where citus could have propagated the creation of a type.
Proactive is almost always a good option. Types are not resource intensive objects, there is no performance overhead of having 100's of types. If you want to use them in a query to represent an intermediate result (which happens in our test suite) they just work.
There is however a moment when proactive type distribution is not beneficial; in transactions where the type is used in a distributed table.
Lets assume the following transaction:
```sql
BEGIN;
CREATE TYPE tt1 AS (a int, b int);
CREATE TABLE t1 AS (a int PRIMARY KEY, b tt1);
SELECT create_distributed_table('t1', 'a');
\copy t1 FROM bigdata.csv
```
Types are node scoped objects; meaning the type exists once per worker. Shards however have best performance when they are created over their own connection. For the type to be visible on all connections it needs to be created and committed before we try to create the shards. Here the just in time situation is most beneficial and follows how we create schema's on the workers. Outside of a transaction block we will just use 1 connection to propagate the creation.
How propagation works
=================
Just in time
-----------
Just in time propagation hooks into the infrastructure introduced in #2882. It adds types as a supported object in `SupportedDependencyByCitus`. This will make sure that any object being distributed by citus that depends on types will now cascade into types. When types are depending them self on other objects they will get created first.
Creation later works by getting the ddl commands to create the object by its `ObjectAddress` in `GetDependencyCreateDDLCommands` which will dispatch types to `CreateTypeDDLCommandsIdempotent`.
For the correct walking of the graph we follow array types, when later asked for the ddl commands for array types we return `NIL` (empty list) which makes that the object will not be recorded as distributed, (its an internal type, dependant on the user type).
Proactive distribution
---------------------
When the user creates a type (composite or enum) we will have a hook running in `multi_ProcessUtility` after the command has been applied locally. Running after running locally makes that we already have an `ObjectAddress` for the type. This is required to mark the type as being distributed.
Keeping the type up to date
====================
For types that are recorded in `pg_dist_object` (eg. `IsObjectDistributed` returns true for the `ObjectAddress`) we will intercept the utility commands that alter the type.
- `AlterTableStmt` with `relkind` set to `OBJECT_TYPE` encapsulate changes to the fields of a composite type.
- `DropStmt` with removeType set to `OBJECT_TYPE` encapsulate `DROP TYPE`.
- `AlterEnumStmt` encapsulates changes to enum values.
Enum types can not be changed transactionally. When the execution on a worker fails a warning will be shown to the user the propagation was incomplete due to worker communication failure. An idempotent command is shown for the user to re-execute when the worker communication is fixed.
Keeping types up to date is done via the executor. Before the statement is executed locally we create a plan on how to apply it on the workers. This plan is executed after we have applied the statement locally.
All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables.
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
Since flattening query may flatten outer joins' columns into coalesce expr that is
in the USING part, and that was not expected before this commit, these queries were
erroring out. It is fixed by this commit with considering coalesce expression as well.
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.
Before this commit, Citus supported INSERT...SELECT queries with
ON CONFLICT or RETURNING clauses only for pushdownable ones, since
queries supported via coordinator were utilizing COPY infrastructure
of PG to send selected tuples to the target worker nodes.
After this PR, INSERT...SELECT queries with ON CONFLICT or RETURNING
clauses will be performed in two phases via coordinator. In the first
phase selected tuples will be saved to the intermediate table which
is colocated with target table of the INSERT...SELECT query. Note that,
a utility function to save results to the colocated intermediate result
also implemented as a part of this commit. In the second phase, INSERT..
SELECT query is directly run on the worker node using the intermediate
table as the source table.
After Fast ALTER TABLE ADD COLUMN with a non-NULL default in PG11, physical heaps might not contain all attributes after a ALTER TABLE ADD COLUMN happens. heap_getattr() returns NULL when the physical tuple doesn't contain an attribute. So we should use heap_deform_tuple() in these cases, which fills in the missing attributes.
Our catalog tables evolve over time, and an upgrade might involve some ALTER TABLE ADD COLUMN commands.
Note that we don't need to worry about postgres catalog tables and we can use heap_getattr() for them, because they only change between major versions.
This also fixes#2453.
With this commit, we all partitioned distributed tables with
replication factor > 1. However, we also have many restrictions.
In summary, we disallow all kinds of modifications (including DDLs)
on the partition tables. Instead, the user is allowed to run the
modifications over the parent table.
The necessity for such a restriction have two aspects:
- We need to acquire shard resource locks appropriately
- We need to handle marking partitions INVALID in case
of any failures. Note that, in theory, the parent table
should also become INVALID, which is too aggressive.
PG11 introduced PROCEDURE concept similar to FUNCTION
Procedure's allow committing/rolling back behavior.
This commmit adds regression tests for procedure calls.
This commit should be reverted once a new PostgreSQL 11 beta is
available: it's due to a bug in the partitioning code which has been
fixed in REL_11_STABLE but (not yet) a released tag.
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.
This table will be used by Citus Enterprise to populate authentication-
related fields in outbound connections; Citus Community lacks support
for this functionality.
After this commit DDL commands honour `citus.multi_shard_modify_mode`.
We preferred using the code-path that executes single task router
queries (e.g., ExecuteSingleModifyTask()) in order not to invent
a new executor that is only applicable for DDL commands that require
sequential execution.
* 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
- changes in ruleutils_11.c is reflected
- vacuum statement api change is handled. We now allow
multi-table vacuum commands.
- some other function header changes are reflected
- api conflicts between PG11 and earlier versions
are handled by adding shims in version_compat.h
- various regression tests are fixed due output and
functionality in PG1
- no change is made to support new features in PG11
they need to be handled by new commit
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 is the first of series of window function work.
We can now support window functions that can be pushed down to workers.
Window function must have distribution column in the partition clause
to be pushed down.
We push down order by to worker query when limit is specified
(with some other additional checks). If the query has an expression
on an aggregate or avg aggregate by itself, and there is an order
by on this particular target we may send wrong order by to worker
query with potential to affect query result.
The fix creates a auxilary target entry in the worker query and
uses that target entry for sorting.
With #1804 (and related PRs), Citus gained the ability to
plan subqueries that are not safe to pushdown.
There are two high-level requirements for pushing down subqueries:
* Individual subqueries that require a merge step (i.e., GROUP BY
on non-distribution key, or LIMIT in the subquery etc). We've
handled such subqueries via #1876.
* Combination of subqueries that are not joined on distribution keys.
This commit aims to recursively plan some of such subqueries to make
the whole query safe to pushdown.
The main logic behind non colocated subquery joins is that we pick
an anchor range table entry and check for distribution key equality
of any other subqueries in the given query. If for a given subquery,
we cannot find distribution key equality with the anchor rte, we
recursively plan that subquery.
We also used a hacky solution for picking relations as the anchor range
table entries. The hack is that we wrap them into a subquery. This is only
necessary since some of the attribute equivalance checks are based on
queries rather than range table entries.
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.
This commit provides the support for window functions in subquery and insert
into select queries. Note that our support for window functions is still limited
because it must have a partition by clause on the distribution key. This commit
makes changes in the files insert_select_planner and multi_logical_planner. The
required tests are also added with files multi_subquery_window_functions.out
and multi_insert_select_window.out.
This change adds support for SAVEPOINT, ROLLBACK TO SAVEPOINT, and RELEASE SAVEPOINT.
When transaction connections are not established yet, savepoints are kept in a stack and sent to the worker when the connection is later established. After establishing connections, savepoint commands are sent as they arrive.
This change fixes#1493 .
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
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.
This commit is intended to be a base for supporting declarative partitioning
on distributed tables. Here we add the following utility functions and their
unit tests:
* Very basic functions including differnentiating partitioned tables and
partitions, listing the partitions
* Generating the PARTITION BY (expr) and adding this to the DDL events
of partitioned tables
* Ability to generate text representations of the ranges for partitions
* Ability to generate the `ALTER TABLE parent_table ATTACH PARTITION
partition_table FOR VALUES value_range`
* Ability to apply add shard ids to the above command using
`worker_apply_inter_shard_ddl_command()`
* Ability to generate `ALTER TABLE parent_table DETACH PARTITION`
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.
Distributed query planning for subquery pushdown is done on the original
query. This prevents the usage of external parameters on the execution.
To overcome this, we manually replace the parameters on the original
query.
* Support for subqueries in WHERE clause
This commit enables subqueries in WHERE clause to be pushed down
by the subquery pushdown logic.
The support covers:
- Correlated subqueries with IN, NOT IN, EXISTS, NOT EXISTS,
operator expressions such as (>, <, =, ALL, ANY etc.)
- Non-correlated subqueries with (partition_key) IN (SELECT partition_key ..)
(partition_key) =ANY (SELECT partition_key ...)
Note that this commit heavily utilizes the attribute equivalence logic introduced
in the 1cb6a34ba8. In general, this commit mostly
adjusts the logical planner not to error out on the subqueries in WHERE clause.
* Improve error checks for subquery pushdown and INSERT ... SELECT
Since we allow subqueries in WHERE clause with the previous commit,
we should apply the same limitations to those subqueries.
With this commit, we do not iterate on each subquery one by one.
Instead, we extract all the subqueries and apply the checks directly
on those subqueries. The aim of this change is to (i) Simplify the
code (ii) Make it close to the checks on INSERT .. SELECT code base.
* Extend checks for unresolved paramaters to include SubLinks
With the presence of subqueries in where clause (i.e., SubPlans on the
query) the existing way for checking unresolved parameters fail. The
reason is that the parameters for SubPlans are kept on the parent plan not
on the query itself (see primnodes.h for the details).
With this commit, instead of checking SubPlans on the modified plans
we start to use originalQuery, where SubLinks represent the subqueries
in where clause. The unresolved parameters can be found on the SubLinks.
* Apply code-review feedback
* Remove unnecessary copying of shard interval list
This commit removes unnecessary copying of shard interval list. Note
that there are no copyObject function implemented for shard intervals.
- There was a crash when the table a shardid belonged to changed during
a session. Instead of crashing (a failed assert) we now throw an error
- Update the isolation test which was crashing to no longer exercise
that code path
- Add a regression test to check that the error is thrown
* Enabling physical planner for subquery pushdown changes
This commit applies the logic that exists in INSERT .. SELECT
planning to the subquery pushdown changes.
The main algorithm is followed as :
- pick an anchor relation (i.e., target relation)
- per each target shard interval
- add the target shard interval's shard range
as a restriction to the relations (if all relations
joined on the partition keys)
- Check whether the query is router plannable per
target shard interval.
- If router plannable, create a task
* Add union support within the JOINS
This commit adds support for UNION/UNION ALL subqueries that are
in the following form:
.... (Q1 UNION Q2 UNION ...) as union_query JOIN (QN) ...
In other words, we currently do NOT support the queries that are
in the following form where union query is not JOINed with
other relations/subqueries :
.... (Q1 UNION Q2 UNION ...) as union_query ....
* Subquery pushdown planner uses original query
With this commit, we change the input to the logical planner for
subquery pushdown. Before this commit, the planner was relying
on the query tree that is transformed by the postgresql planner.
After this commit, the planner uses the original query. The main
motivation behind this change is the simplify deparsing of
subqueries.
* Enable top level subquery join queries
This work enables
- Top level subquery joins
- Joins between subqueries and relations
- Joins involving more than 2 range table entries
A new regression test file is added to reflect enabled test cases
* Add top level union support
This commit adds support for UNION/UNION ALL subqueries that are
in the following form:
.... (Q1 UNION Q2 UNION ...) as union_query ....
In other words, Citus supports allow top level
unions being wrapped into aggregations queries
and/or simple projection queries that only selects
some fields from the lower level queries.
* Disallow subqueries without a relation in the range table list for subquery pushdown
This commit disallows subqueries without relation in the range table
list. This commit is only applied for subquery pushdown. In other words,
we do not add this limitation for single table re-partition subqueries.
The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.
* Disallow subqueries without a relation in the range table list for INSERT .. SELECT
This commit disallows subqueries without relation in the range table
list. This commit is only applied for INSERT.. SELECT queries.
The reasoning behind this limitation is that if we allow pushing down
such queries, the result would include (shardCount * expectedResults)
where in a non distributed world the result would be (expectedResult)
only.
* Change behaviour of subquery pushdown flag (#1315)
This commit changes the behaviour of the citus.subquery_pushdown flag.
Before this commit, the flag is used to enable subquery pushdown logic. But,
with this commit, that behaviour is enabled by default. In other words, the
flag is now useless. We prefer to keep the flag since we don't want to break
the backward compatibility. Also, we may consider using that flag for other
purposes in the next commits.
* Require subquery_pushdown when limit is used in subquery
Using limit in subqueries may cause returning incorrect
results. Therefore we allow limits in subqueries only
if user explicitly set subquery_pushdown flag.
* Evaluate expressions on the LIMIT clause (#1333)
Subquery pushdown uses orignal query, the LIMIT and OFFSET clauses
are not evaluated. However, logical optimizer expects these expressions
are already evaluated by the standard planner. This commit manually
evaluates the functions on the logical planner for subquery pushdown.
* Better format subquery regression tests (#1340)
* Style fix for subquery pushdown regression tests
With this commit we intented a more consistent style for the
regression tests we've added in the
- multi_subquery_union.sql
- multi_subquery_complex_queries.sql
- multi_subquery_behavioral_analytics.sql
* Enable the tests that are temporarily commented
This commit enables some of the regression tests that were commented
out until all the development is done.
* Fix merge conflicts (#1347)
- Update regression tests to meet the changes in the regression
test output.
- Replace Ifs with Asserts given that the check is already done
- Update shard pruning outputs
* Add view regression tests for increased subquery coverage (#1348)
- joins between views and tables
- joins between views
- union/union all queries involving views
- views with limit
- explain queries with view
* Improve btree operators for the subquery tests
This commit adds the missing comprasion for subquery composite key
btree comparator.
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 DropShards function started to use new connection API. DropShards
function is used by DROP TABLE, master_drop_all_shards and master_apply_delete_command,
therefore all of these functions now support transactional operations. In DropShards
function, if we cannot reach a node, we mark shard state of related placements as
FILE_TO_DELETE and continue to drop remaining shards; however if any error occurs after
establishing the connection, we ROLLBACK whole operation.
With this change, we start to delete placement of reference tables at given worker node
after master_remove_node UDF call. We remove placement metadata at master node but we do
not drop actual shard from the worker node. There are two reasons for that decision,
first, it is not critical to DROP the shards in the workers because Citus will ignore them
as long as node is removed from cluster and if we add that node back to cluster we will
DROP and recreate all reference tables. Second, if node is unreachable, it becomes
complicated to cover failure cases and have a transaction support.
Enables use views within distributed queries.
User can create and use a view on distributed tables/queries
as he/she would use with regular queries.
After this change router queries will have full support for views,
insert into select queries will support reading from views, not
writing into. Outer joins would have a limited support, and would
error out at certain cases such as when a view is in the inner side
of the outer join.
Although PostgreSQL supports writing into views under certain circumstances.
We disallowed that for distributed views.
With this change, we start to replicate all reference tables to the new node when new node
is added to the cluster with master_add_node command. We also update replication factor
of reference table's colocation group.