Commit Graph

764 Commits (test/adaptive_executor_repartition)

Author SHA1 Message Date
Jelte Fennema 78e495e030
Add shouldhaveshards to pg_dist_node (#2960)
This is an improvement over #2512.

This adds the boolean shouldhaveshards column to pg_dist_node. When it's false, create_distributed_table for new collocation groups will not create shards on that node. Reference tables will still be created on nodes where it is false.
2019-10-22 16:47:16 +02:00
Jelte Fennema 7abedc38b0
Support subqueries in HAVING (#3098)
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 #520
Fixes #756
Closes #2047
2019-10-16 16:40:14 +02:00
Onur TIRTIR d5f83dc110
Refactor range table walkers (#3109) 2019-10-16 01:20:49 +03:00
SaitTalhaNisanci 94a7e6475c
Remove copyright years (#2918)
* Update year as 2012-2019

* Remove copyright years
2019-10-15 17:44:30 +03:00
Philip Dubé 74cb168205 Remove Postgres 10 support 2019-10-11 21:56:56 +00:00
Philip Dubé dd490b6376 Cache whether an object is in pg_dist_object. Avoids redundant lookups for non-distributed objects 2019-10-10 14:50:38 +00:00
Onder Kalaci 3be72ce42f Make sure that distributed functions always have the correct user
Objectives:

(a) both super user and regular user should have the correct owner for the function on the worker
(b) The transactional semantics would work fine for both super user and regular user
(c) non-super-user and non-function owner would get a reasonable error message if tries to distribute the function

Co-authored-by: @serprex
2019-10-04 21:38:49 +00:00
Hadi Moshayedi 217db2a03e Don't block for locks in SyncMetadataToNodes() 2019-10-03 16:53:36 -07:00
Philip Dubé 89d35e9692 Attempt to force custom plans for prepared statements when trying to delegate function calls
We discern between PARAM_EXEC & PARAM_EXTERN:
d52eaa0948/src/include/nodes/primnodes.h (L211)
According to primnodes.h we should only run into PARAM_EXEC or PARAM_EXTERN
2019-09-30 23:49:14 +00:00
Hadi Moshayedi 5e97e5c98e Don't push down queries when in subqueries/ctes 2019-09-30 14:22:05 -07:00
Marco Slot 35bef0f3db Avoid caching connections from backends that servicei internal connections 2019-09-28 08:32:10 +02:00
Nils Dijk 01b26cf91a
Disallow distributed functions for functions depending on an extension (#3049)
DESCRIPTION: Disallow distributed functions for functions depending on an extension

Functions depending on an extension cannot (yet) be distributed by citus. If we would allow this it would cause issues with our dependency following mechanism as we stop following objects depending on an extension.

By not allowing functions to be distributed when they depend on an extension as well as not allowing to make distributed functions depend on an extension we won't break the ability to add new nodes. Allowing functions depending on extensions to be distributed at the moment could cause problems in that area.
2019-09-30 15:19:47 +02:00
Nils Dijk 473cbc0115
Propagate CREATE OR REPLACE FUNCTION to workers for distributed functions (#3043)
DESCRIPTION: Propagate CREATE OR REPLACE FUNCTION

Distributed functions could be replaced, which should be propagated to the workers to keep the function in sync between all nodes.

Due to the complexity of deparsing the `CreateFunctionStmt` we actually produce the plan during the processing phase of our utilityhook. Since the changes have already been made in the catalog tables we can reuse `pg_get_functiondef` to get us the generated `CREATE OR REPLACE` sql.
2019-09-30 12:41:17 +02:00
Nils Dijk 9c2c50d875
Hookup function/procedure deparsing to our utility hook (#3041)
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.
2019-09-27 22:06:49 +02:00
Philip Dubé 363409a0c2 Propagate REINDEX TABLE & REINDEX INDEX 2019-09-27 18:14:53 +00:00
Hanefi Onaldi 66b9f2e887 Deparsing and qualifiying for FUNCTION/PROCEDURE statements (#3014)
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.)
2019-09-27 19:02:52 +02:00
Marco Slot 2868e02a3d Implement SELECT function call delegation.
When a function is marked as colocated with a distributed table,
we try delegating queries of kind "SELECT func(...)" to workers.

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

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

Co-authored-by: Marco Slot <marco@citusdata.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
2019-09-27 09:13:25 -07:00
Marco Slot ca478defeb Deparse CALL statement instead of using original query string 2019-09-24 17:31:09 +00:00
Marco Slot e269d990c9 Cast the distribution argument value when possible 2019-09-24 17:31:09 +00:00
Philip Dubé 432a8ef85b Hadi's feedback
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
2019-09-24 17:31:09 +00:00
Philip Dubé bc1ad67eb5 Distribute CALL on distributed procedures to metadata workers
Lots taken from https://github.com/citusdata/citus/pull/2829
2019-09-24 17:31:09 +00:00
Onder Kalaci d37745bfc7 Sync metadata to worker nodes after create_distributed_function
Since the distributed functions are useful when the workers have
metadata, we automatically sync it.

Also, after master_add_node(). We do it lazily and let the deamon
sync it. That's mainly because the metadata syncing cannot be done
in transaction blocks, and we don't want to add lots of transactional
limitations to master_add_node() and create_distributed_function().
2019-09-23 18:30:53 +02:00
Marco Slot 5f23b951c7 Support serial and smallserial when syncing metadata 2019-09-23 17:39:21 +02:00
Marco Slot e58d76c5f6 Fix assert failure in bare SELECT FROM reference table FOR UPDATE in MX 2019-09-23 17:00:09 +02:00
Hanefi Onaldi ed11b9590c
Add distributed func creation queries in dependency replication logic 2019-09-18 20:07:45 +03:00
Hadi Moshayedi 76f3933b05 Add metadatasynced, and sync on master_update_node()
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
2019-09-18 09:32:54 -07:00
Nils Dijk db5d03931d
Feature disable object propagation (#2986)
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.
2019-09-18 17:16:22 +02:00
Nils Dijk 2b7f5552c8
Fix: rename remote type on conflict (#2983)
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.
2019-09-17 18:54:10 +02:00
Nils Dijk 0a3152d09c
Add feature flag to turn off create type propagation (#2982)
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.
2019-09-17 15:50:06 +02:00
Philip Dubé 964020097d Merge two conflicting pg_dist_object headers 2019-09-16 19:19:21 +00:00
Onder Kalaci cde6b02858 Add columns to pg_dist_object for distributed functions
This PR simply adds the columns to pg_dist_object and
implements the necessary metadata changes to keep track of
distribution argument of the functions/procedures.
2019-09-16 17:28:04 +02:00
Philip Dubé 492d1b2cba ActivePrimaryNodeList: add lockMode parameter 2019-09-13 17:44:56 +00:00
Nils Dijk 2879689441
Distribute Types to worker nodes (#2893)
DESCRIPTION: Distribute Types to worker nodes

When to propagate
==============

There are two logical moments that types could be distributed to the worker nodes
 - When they get used ( just in time distribution )
 - When they get created ( proactive distribution )

The just in time distribution follows the model used by how schema's get created right before we are going to create a table in that schema, for types this would be when the table uses a type as its column.

The proactive distribution is suitable for situations where it is benificial to have the type on the worker nodes directly. They can later on be used in queries where an intermediate result gets created with a cast to this type.

Just in time creation is always the last resort, you cannot create a distributed table before the type gets created. A good example use case is; you have an existing postgres server that needs to scale out. By adding the citus extension, add some nodes to the cluster, and distribute the table. The type got created before citus existed. There was no moment where citus could have propagated the creation of a type.

Proactive is almost always a good option. Types are not resource intensive objects, there is no performance overhead of having 100's of types. If you want to use them in a query to represent an intermediate result (which happens in our test suite) they just work.

There is however a moment when proactive type distribution is not beneficial; in transactions where the type is used in a distributed table.

Lets assume the following transaction:

```sql
BEGIN;
CREATE TYPE tt1 AS (a int, b int);
CREATE TABLE t1 AS (a int PRIMARY KEY, b tt1);
SELECT create_distributed_table('t1', 'a');
\copy t1 FROM bigdata.csv
```

Types are node scoped objects; meaning the type exists once per worker. Shards however have best performance when they are created over their own connection. For the type to be visible on all connections it needs to be created and committed before we try to create the shards. Here the just in time situation is most beneficial and follows how we create schema's on the workers. Outside of a transaction block we will just use 1 connection to propagate the creation.

How propagation works
=================

Just in time
-----------

Just in time propagation hooks into the infrastructure introduced in #2882. It adds types as a supported object in `SupportedDependencyByCitus`. This will make sure that any object being distributed by citus that depends on types will now cascade into types. When types are depending them self on other objects they will get created first.

Creation later works by getting the ddl commands to create the object by its `ObjectAddress` in `GetDependencyCreateDDLCommands` which will dispatch types to `CreateTypeDDLCommandsIdempotent`.

For the correct walking of the graph we follow array types, when later asked for the ddl commands for array types we return `NIL` (empty list) which makes that the object will not be recorded as distributed, (its an internal type, dependant on the user type).

Proactive distribution
---------------------

When the user creates a type (composite or enum) we will have a hook running in `multi_ProcessUtility` after the command has been applied locally. Running after running locally makes that we already have an `ObjectAddress` for the type. This is required to mark the type as being distributed.

Keeping the type up to date
====================

For types that are recorded in `pg_dist_object` (eg. `IsObjectDistributed` returns true for the `ObjectAddress`) we will intercept the utility commands that alter the type.
 - `AlterTableStmt` with `relkind` set to `OBJECT_TYPE` encapsulate changes to the fields of a composite type.
 - `DropStmt` with removeType set to `OBJECT_TYPE` encapsulate `DROP TYPE`.
 - `AlterEnumStmt` encapsulates changes to enum values.
    Enum types can not be changed transactionally. When the execution on a worker fails a warning will be shown to the user the propagation was incomplete due to worker communication failure. An idempotent command is shown for the user to re-execute when the worker communication is fixed.

Keeping types up to date is done via the executor. Before the statement is executed locally we create a plan on how to apply it on the workers. This plan is executed after we have applied the statement locally.

All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables.
2019-09-13 17:46:07 +02:00
Jelte Fennema d6deb062aa Add shard rebalancer stubs 2019-09-12 16:40:25 +02:00
Jelte Fennema 58012054c9 Add an extra advisory lock tag class 2019-09-12 16:40:25 +02:00
Jelte Fennema eb7e45d556 Make LookupNodeForGroup extern 2019-09-12 16:40:25 +02:00
Jelte Fennema de5174f763 include postgres.h into some of our .h files to silence warnings 2019-09-12 16:40:25 +02:00
Onder Kalaci 0b0c779c77 Introduce the concept of Local Execution
/*
 * local_executor.c
 *
 * The scope of the local execution is locally executing the queries on the
 * shards. In other words, local execution does not deal with any local tables
 * that are not shards on the node that the query is being executed. In that sense,
 * the local executor is only triggered if the node has both the metadata and the
 * shards (e.g., only Citus MX worker nodes).
 *
 * The goal of the local execution is to skip the unnecessary network round-trip
 * happening on the node itself. Instead, identify the locally executable tasks and
 * simply call PostgreSQL's planner and executor.
 *
 * The local executor is an extension of the adaptive executor. So, the executor uses
 * adaptive executor's custom scan nodes.
 *
 * One thing to note that Citus MX is only supported with replication factor = 1, so
 * keep that in mind while continuing the comments below.
 *
 * On the high level, there are 3 slightly different ways of utilizing local execution:
 *
 * (1) Execution of local single shard queries of a distributed table
 *
 *      This is the simplest case. The executor kicks at the start of the adaptive
 *      executor, and since the query is only a single task the execution finishes
 *      without going to the network at all.
 *
 *      Even if there is a transaction block (or recursively planned CTEs), as long
 *      as the queries hit the shards on the same, the local execution will kick in.
 *
 * (2) Execution of local single queries and remote multi-shard queries
 *
 *      The rule is simple. If a transaction block starts with a local query execution,
 *      all the other queries in the same transaction block that touch any local shard
 *      have to use the local execution. Although this sounds restrictive, we prefer to
 *      implement in this way, otherwise we'd end-up with as complex scenarious as we
 *      have in the connection managements due to foreign keys.
 *
 *      See the following example:
 *      BEGIN;
 *          -- assume that the query is executed locally
 *          SELECT count(*) FROM test WHERE key = 1;
 *
 *          -- at this point, all the shards that reside on the
 *          -- node is executed locally one-by-one. After those finishes
 *          -- the remaining tasks are handled by adaptive executor
 *          SELECT count(*) FROM test;
 *
 *
 * (3) Modifications of reference tables
 *
 *		Modifications to reference tables have to be executed on all nodes. So, after the
 *		local execution, the adaptive executor keeps continuing the execution on the other
 *		nodes.
 *
 *		Note that for read-only queries, after the local execution, there is no need to
 *		kick in adaptive executor.
 *
 *  There are also few limitations/trade-offs that is worth mentioning. First, the
 *  local execution on multiple shards might be slow because the execution has to
 *  happen one task at a time (e.g., no parallelism). Second, if a transaction
 *  block/CTE starts with a multi-shard command, we do not use local query execution
 *  since local execution is sequential. Basically, we do not want to lose parallelism
 *  across local tasks by switching to local execution. Third, the local execution
 *  currently only supports queries. In other words, any utility commands like TRUNCATE,
 *  fails if the command is executed after a local execution inside a transaction block.
 *  Forth, the local execution cannot be mixed with the executors other than adaptive,
 *  namely task-tracker, real-time and router executors. Finally, related with the
 *  previous item, COPY command cannot be mixed with local execution in a transaction.
 *  The implication of that any part of INSERT..SELECT via coordinator cannot happen
 *  via the local execution.
 */
2019-09-12 11:51:25 +02:00
Nils Dijk 936d546a3c
Refactor Ensure Schema Exists to Ensure Dependecies Exists (#2882)
DESCRIPTION: Refactor ensure schema exists to dependency exists

Historically we only supported schema's as table dependencies to be created on the workers before a table gets distributed. This PR puts infrastructure in place to walk pg_depend to figure out which dependencies to create on the workers. Currently only schema's are supported as objects to create before creating a table.

We also keep track of dependencies that have been created in the cluster. When we add a new node to the cluster we use this catalog to know which objects need to be created on the worker.

Side effect of knowing which objects are already distributed is that we don't have debug messages anymore when creating schema's that are already created on the workers.
2019-09-04 14:10:20 +02:00
Philip Dubé 28d964240f Remove CheckForUpdates
https://reports.citusdata.com/v1/releases/latest
We haven't updated the version CheckForUpdates sees since 7.1.0
2019-09-03 21:11:25 +00:00
Jelte Fennema cbecf97c84
Move tuplestore setup to a helper function (#2898)
* Add tuplestore helpers

* More detailed error messages in tuplestore

* Add CreateTupleDescCopy to SetupTuplestore

* Use new SetupTuplestore helper function

* Remove unnecessary copy

* Remove comment about undefined behaviour
2019-08-27 09:11:08 +02:00
Philip Dubé 6b0d8ed83d SortList in FinalizedShardPlacementList, makes 3 failure tests consistent between 11/12 2019-08-22 19:30:56 +00:00
Philip Dubé e5cd298a98 pg12 revised layout of FunctionCallInfoData
See a9c35cf85c

clang raises a warning due to FunctionCall2InfoData technically being variable sized
This is fine, as the struct is the size we want it to be. So silence the warning
2019-08-22 19:02:35 +00:00
Philip Dubé be3285828f Collations matter for hashing strings in pg12
See https://www.postgresql.org/docs/12/collation.html#COLLATION-NONDETERMINISTIC
2019-08-22 18:58:37 +00:00
Philip Dubé fe10ca453d Implement FileCompat to abstract pg12 requiring API consumer to track file offsets 2019-08-22 18:57:47 +00:00
Philip Dubé 018ad1c58e pg12: version_compat.h, tuples, oids, misc 2019-08-22 18:57:23 +00:00
Philip Dubé 68c4b71f93 Fix up includes with pg12 changes 2019-08-22 18:56:21 +00:00
Philip Dubé fbc3e346e8 ruleutils_12.c
Produced this file by copying ruleutils_11.c,
then comparing postgres ruleutils.c changes between REL_11_STABLE & REL_12_STABLE
2019-08-22 18:56:05 +00:00
Hadi Moshayedi a5b087c89b Support FKs between reference tables 2019-08-21 16:11:27 -07:00
Philip Dubé f4b90419ae Raise an error when REINDEX TABLE or INDEX is invoked on a distributed relation 2019-08-21 17:03:14 +00:00
Philip Dubé f4e513b3d4 Introduce citus.single_shard_commit_protocol for if users want 1PC on writes to replicas 2019-08-15 18:49:40 +00:00
Philip Dubé cd951fa9ca Avoid multiple pg_dist_colocation records being created for reference tables
master_deactivate_node is updated to decrement the replication factor
Otherwise deactivation could have create_reference_table produce a second record

UpdateColocationGroupReplicationFactor is renamed UpdateColocationGroupReplicationFactorForReferenceTables
& the implementation looks up the record based on distributioncolumntype == InvalidOid, rather than by id
Otherwise the record's replication factor fails to be maintained when there are no reference tables
2019-08-13 17:21:02 +00:00
Hadi Moshayedi 009d8b7401 Some cleanup 2019-08-12 15:38:52 -07:00
Onder Kalaci 060ac11476 Do not record relation accessess unnecessarily
Before this commit, we've recorded the relation accesses in 3 different
places
    - FindPlacementListConnection         -- applies all executor in tx block
    - StartPlacementExecutionOnSession()  -- adaptive executor only
    - StartPlacementListConnection()      -- router/real-time only

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

Benchmarks:

```
1+8 c5.4xlarge cluster

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

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

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

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

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

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

| Test  | Citus 8.2.2  |  Citus 8.3.1   | Citus 8.3.2 (this branch)  | Citus 8.3.1 (FKEYs disabled via GUC)  |
| ------------- | ------------- | ------------- |------------- | ------------- |
|multi_shard |  2370.083 ms  |3605.040 ms |1324.094 ms |1247.255 ms  |
| single_shard  | 85.338 ms  |120.934 ms  |73.216 ms  | 78.765 ms |
| mix  | 2434.459 ms | 3727.080 ms  |1306.456 ms  | 1280.326 ms |
2019-08-08 18:42:08 +02:00
Onder Kalaci 35ee896f3d Get rid of an unnecessary parameter
targetPoolSize parameter for ExecuteUtilityTaskListWithoutResults
becomes obsolete, just remove it.
2019-08-07 19:35:56 +02:00
Onder Kalaci b2e01d0745 Refactor switching to sequential mode
We don't need to wait until the execution. As soon as we realize
that we need sequential execution, we should do it.
2019-08-07 19:35:56 +02:00
Philip Dubé b77c52f95b PlanRouterQuery: don't store list of list of shard intervals in relationShardList 2019-08-02 14:08:57 +00:00
Marco Slot 4444d92dbc Set initial pool size to cached connection count 2019-07-23 20:40:32 +02:00
Philip Dubé 0915027389 DistributedPlan: replace operation with modLevel
This causes no behaviorial changes, only organizes better to implement modifying CTEs

Also rename ExtactInsertRangeTableEntry to ExtractResultRelationRTE,
as the source of this function didn't match the documentation

Remove Task's upsertQuery in favor of ROW_MODIFY_NONCOMMUTATIVE

Split up AcquireExecutorShardLock into more internal functions

Tests: Normalize multi_reference_table multi_create_table_constraints
2019-07-16 13:58:18 -07:00
Marco Slot 07d2266e11 Fix RESET and other types of SET 2019-07-05 19:30:48 +02:00
Önder Kalacı 40da78c6fd
Introduce the adaptive executor (#2798)
With this commit, we're introducing the Adaptive Executor. 


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

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



All the commits involved here:
* Initial unified executor prototype

* Latest changes

* Fix rebase conflicts to master branch

* Add missing variable for assertion

* Ensure that master_modify_multiple_shards() returns the affectedTupleCount

* Adjust intermediate result sizes

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

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

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

That might be useful under certain circumstances:

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

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

* For parameters to be resolved before using them

For the details, see PostgreSQL's copyParamList()

* Unified executor sorts the returning output

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

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

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

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

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

* Fix 2PC conditions for DDL tasks

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

* Support foreign keys to reference tables in unified executor

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

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

* Make sure to close the failed connections after the execution

* Improve comments

* Fix savepoints in unified executor.

* Rebuild the WaitEventSet only when necessary

* Unclaim connections on all errors.

* Improve failure handling for unified executor

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

Improve the logic to start a transaction block for distributed transactions

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

* Move waitEventSet to DistributedExecution

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

* Fix multi_insert_select test outputs

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

* Fix subeury_and_cte test

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

which is a lot clearer to the user.

* Fix intermediate_results test outputs

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

which makes a lot more sense.

* Fix multi_function_in_join test

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

* Fix multi_query_directory_cleanup test

The unified executor does not create any intermediate files.

* Fix with_transactions test

A test case that just started to work fine

* Fix multi_router_planner test outputs

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

which is a lot more clearer for the users

* Fix multi_router_planner_fast_path test

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

which is a lot more clearer for the users

* Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block

* Fix ordering in isolation_multi_shard_modify_vs_all

* Add executor locks to unified executor

* Make sure to allocate enought WaitEvents

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

* Fix rebase conflicts for master rebase

* Make sure that TRUNCATE relies on unified executor

* Implement true sequential execution for multi-row INSERTS

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

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

* Remove SESSION_LIFESPAN flag in unified_executor

* Apply failure test updates

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

* Unified executor honors citus.node_connection_timeout

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

And, as a side effect this fixes failure_connection_establishment
test.

* Properly increment/decrement pool size variables

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

* insert_select_executor goes through unified executor.

* Add missing file for task tracker

* Modify ExecuteTaskListExtended()'s signature

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

* Take partition locks correctly in unified executor

* Alternative implementation for force_max_query_parallelization

* Fix compile warnings in unified executor

* Fix style issues

* Decrement idleConnectionCount when idle connection is lost

* Always rebuild the wait event sets

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

* Make sure to allocate enough sized waitEventSet

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

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

* Implement SELECT FOR UPDATE on reference tables

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

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

* SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled

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

* Fix build error

* Fix the newConnectionCount calculation in ManageWorkerPool

* Fix rebase conflicts

* Fix minor test output differences

* Fix citus indent

* Remove duplicate sorts that is added with rebase

* Create distributed table via executor

* Fix wait flags in CheckConnectionReady

* failure_savepoints output for unified executor.

* failure_vacuum output (pg 10) for unified executor.

* Fix WaitEventSetWait timeout in unified executor

* Stabilize failure_truncate test output

* Add an ORDER BY to multi_upsert

* Fix regression test outputs after rebase to master

* Add executor.c comment

* Rename executor.c to adaptive_executor.c

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

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

* Implement a proper custom scan node for adaptive executor

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

* Add slow start interval to the executor

* Expose max_cached_connection_per_worker to user

* Do not start slow when there are cached connections

* Consider ExecutorSlowStartInterval in NextEventTimeout

* Fix memory issues with ReceiveResults().

* Disable executor via TaskExecutorType

* Make sure to execute the tests with the other executor

* Use task_executor_type to enable-disable adaptive executor

* Remove useless code

* Adjust the regression tests

* Add slow start regression test

* Rebase to master

* Fix test failures in adaptive executor.

* Rebase to master - 2

* Improve comments & debug messages

* Set force_max_query_parallelization in isolation_citus_dist_activity

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

* Adjust the default pool size

* Expand description of max_adaptive_executor_pool_size GUC

* Update warnings in FinishRemoteTransactionCommit()

* Improve session clean up at the end of execution

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

* Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all

* Add more ORDER BYs to multi_mx_partitioning
2019-06-28 14:04:40 +02:00
Hanefi Onaldi 7e8fd49b94 Create Schemas as superuser on all shard/table creation UDFs
- All the schema creations on the workers will now be  via superuser connections
- If a shard is being repaired or a shard is replicated, we will create the
  schema only in the relevant worker; and in all the other cases where a schema
  creation is needed, we will block operations until we ensure the schema exists
  in all the workers
2019-06-26 17:12:28 +02:00
Nils Dijk 5df1b49bed
Feature: optionally force master_update_node during failover (#2773)
When `master_update_node` is called to update a node's location it waits for appropriate locks to become available. This is useful during normal operation as new operations will be blocked till after the metadata update while running operations have time to finish.

When `master_update_node` is called after a node failure it is less useful to wait for running operations to finish as they can't. The lock being held indicates an operation that once attempted to commit will fail as the machine already failed. Now the downside is the failover is postponed till the termination point of the operation. This has been observed by users to take a significant amount of time causing the rest of the system to be observed unavailable.

With this patch it is possible in such situations to invoke `master_update_node` with 2 optional arguments:
 - `force` (bool defaults to `false`): When called with true the update of the metadata will be forced to proceed by terminating conflicting backends. A cancel is not enough as the backend might be in idle time (eg. an interactive session, or going back and forth between an appliaction), therefore a more intrusive solution of termination is used here.
 - `lock_cooldown` (int defaults to `10000`): This is the time in milliseconds before conflicting backends are terminated. This is to allow the backends to finish cleanly before terminating them. This allows the user to set an upperbound to the expected time to complete the metadata update, eg. performing the failover.

The functionality is implemented by spawning a background worker that has the task of helping a certain backend in acquiring its locks. The backend is either terminated on successful execution of the metadata update, or once the memory context of the expression gets reset, eg. on a cancel of the statement.
2019-06-21 12:03:15 +02:00
Jason Petersen d4e1172247 Implement propagation of SET LOCAL commands
Adds support for propagation of SET LOCAL commands to all workers
involved in a query. For now, SET SESSION (i.e. plain SET) is not
supported whatsoever, though this code is intended as somewhat of a
base for implementing such support in the future.

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

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

A new GUC (citus.propagate_set_commands) has been added to control this
behavior. Though the code suggests the valid settings are 'none', 'local',
'session', and 'all', only 'none' (the default) and 'local' are presently
implemented: attempting to use other values will result in an error.
2019-06-20 16:15:43 -07:00
Jason Petersen 1dec6c5163 Change BeginCoordinatedTransaction to internal linkage
It's only ever called from a single file, so having it be extern didn't
make a whole lot of sense.
2019-06-20 13:44:06 -07:00
Hadi Moshayedi 4bbae02778 Make COPY compatible with unified executor. 2019-06-20 19:53:40 +02:00
Hadi Moshayedi 2e6d04df7b Refactor ExecuteModifyTasksSequentially. 2019-06-20 18:38:57 +02:00
Onder Kalaci 4fd1fcbbef Refactor shard creation logic
This is a preperation for the new executor, where creating shards
would go through the executor. So, explicitly generate the commands
for further processing.
2019-06-19 10:03:58 +02:00
Hadi Moshayedi 85325e0098 Refactor ScanStateGetExecutorState into its own function. 2019-06-05 09:16:43 -07:00
Hadi Moshayedi 0b01c59fa6 Refactor ScanStateGetTupleDescriptor() into a function. 2019-06-04 15:19:49 -07:00
Hadi Moshayedi 8e2d328530 Search all outer node levels for lateral join params. 2019-06-04 10:14:05 -07:00
Marco Slot bb3a96eacb Cache a configurable number of connections at xact end 2019-05-29 13:24:31 +02:00
exialin 59e54de54d Minor code clean-up 2019-05-24 14:26:26 +02:00
Hanefi Onaldi 4d737177e6
Remove redundant active placement filters and unneded sort operations
If a query is router executable, it hits a single shard and therefore has a
single task associated with it. Therefore there is no need to sort the task list
that has a single element.

Also we already have a list of active shard placements, sending it in param
and reuse it.
2019-05-24 14:16:50 +03:00
Hanefi Onaldi b935dfb8c8
Cleanup deleted function declaration 2019-05-24 14:04:26 +03:00
Hadi Moshayedi b5c0ca45f1 Remove stopOnFailure flag from EndRemoteCopy() 2019-05-11 06:18:34 -07:00
Onder Kalaci 495b6e9b62 Refactor Parallel Relation Access Recording
Instead of scattering the code around, we move all the
logic into a single function.

This will help supporting foreign keys to reference tables
in the unified executor with a single line of change, just
calling this function.
2019-05-02 18:12:33 +03:00
Hadi Moshayedi aafd22dffa Fix savepoint rollback for INSERT INTO ... SELECT. 2019-05-01 09:33:43 -07:00
Hadi Moshayedi b69a762e0b Fix savepoint rollback after multi-shard update failure. 2019-05-01 09:33:43 -07:00
Jason Petersen 71d5d1c865 Enable variable shadowing warnings; fix all
Rather than wait for another place like the previous commit to bite us,
I think we should turn on this warning.
2019-04-30 13:24:25 -06:00
Onder Kalaci 004f28e18c Sort output of RETURNING
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.
2019-04-24 11:51:19 +03:00
Nils Dijk feaac69769
Implementation for asycn FinishConnectionListEstablishment (#2584) 2019-03-22 17:30:42 +01:00
Jason Petersen a2c6f596f9 Address code review comments 2019-03-21 11:59:52 -06:00
Jason Petersen 00d836e5a3 alloc non-global conn. params in provided context
Having DATA-segment string literals made blindly freeing the keywords/
values difficult, so I've switched to allocating all in the provided
context; because of this (and with the knowledge of the end point of
the global parameters), we can safely pfree non-global parameters when
we come across an invalid connection parameter entry.
2019-03-21 00:03:35 -06:00
Marco Slot ee6a0b6943 Speed up RTE walkers
Do it in two ways (a) re-use the rte list as much as possible instead of
re-calculating over and over again (b) Limit the recursion to the relevant
parts of the query tree
2019-03-20 12:14:46 +03:00
Marco Slot 5ff1821411 Cache the current database name
Purely for performance reasons.
2019-03-20 12:14:46 +03:00
Marco Slot 0ea4e52df5 Add nodeId to shardPlacements and use it for shard placement comparisons
Before this commit, shardPlacements were identified with shardId, nodeName
and nodeport. Instead of using nodeName and nodePort, we now use nodeId
since it apparently has performance benefits in several places in the
code.
2019-03-20 12:14:46 +03:00
Marco Slot f2abf2b8e5 Functions are treated as transaction blocks 2019-03-15 16:34:08 -06:00
Hadi Moshayedi a9e6d06a98 Skip execution of ALTER TABLE constraint checks on the coordinator 2019-03-14 15:40:56 -07:00
Hadi Moshayedi f4d3b94e22
Fix some of the casts for groupId (#2609)
A small change which partially addresses #2608.
2019-03-05 12:06:44 -08:00
Onder Kalaci f706772b2f Round-robin task assignment policy relies on local transaction id
Before this commit, round-robin task assignment policy was relying
on the taskId. Thus, even inside a transaction, the tasks were
assigned to different nodes. This was especially problematic
while reading from reference tables within transaction blocks.
Because, we had to expand the distributed transaction to many
nodes that are not necessarily already in the distributed transaction.
2019-02-22 19:26:38 +03:00
Onder Kalaci f144bb4911 Introduce fast path router planning
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.
2019-02-21 13:27:01 +03:00
Hanefi Onaldi 825666f912
Query samples in docs and better errors 2019-02-04 19:20:02 +03:00
Hanefi Onaldi 574b071113
Add wrapper function introduced in PG11 for compatibility 2019-02-04 19:20:02 +03:00
Jason Petersen 339e6e661e
Remove 9.6 (#2554)
Removes support and code for PostgreSQL 9.6

cr: @velioglu
2019-01-16 13:11:24 -07:00
Marco Slot 1656b519c4 Plan outer joins through pushdown planning 2019-01-05 20:55:27 +01:00
Marco Slot 3ff2b47366 Restrict visibility of get_*_active_transactions functions to pg_monitor 2018-12-19 18:32:42 +01:00
velioglu 3e0cff94a6 Add FunctionOidExtended function 2018-12-10 11:59:41 +03:00
Nils Dijk 4af40eee76 Enable SSL by default during installation of citus 2018-12-07 11:23:19 -07:00
velioglu 8764a19464 Adds support for disabling hash agg with hll functions on coordinator query 2018-12-07 18:49:25 +03:00
Onder Kalaci 621ccf3946 Ensure to use initialized MaxBackends
Postgresql loads shared libraries before calculating MaxBackends.
However, Citus relies on MaxBackends being set. Thus, with this
commit we use the same steps to calculate MaxBackends while
Citus is being loaded (e.g., PG_Init is called).

Note that this is safe since all the elements that are used to
calculate MaxBackends are PGC_POSTMASTER gucs and a constant
value.
2018-12-03 13:25:51 +03:00
Onder Kalaci b6ebd791a6 Sort task list for multi-task explain outputs
This is purely for ensuring that regression tests do not randomly fail.
2018-11-30 11:19:37 -07:00
Marco Slot 8893cc141d Support INSERT...SELECT with ON CONFLICT or RETURNING via coordinator
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.
2018-11-30 15:29:12 +03:00
Marco Slot 8e93fe5870 Check schema owner in task_tracker_assign_task 2018-11-23 11:05:09 +01:00
Marco Slot 6aa5592e52 Add user ID suffix to intermediate files in re-partition jobs 2018-11-23 08:36:11 +01:00
Marco Slot a59bf31c76 Use worker_execute_sql_task UDF in task-tracker executor 2018-11-22 18:15:33 +01:00
Marco Slot caf402d506 COPY to a task file no longer switches to superuser 2018-11-22 18:15:33 +01:00
Onder Kalaci 052ba21b19 Make sure to prevent unauthorized users to drop sequences in Citus MX 2018-11-15 18:08:04 +03:00
Nils Dijk f9520be011
Round robin queries to reference tables with task_assignment_policy set to `round-robin` (#2472)
Description: Support round-robin `task_assignment_policy` for queries to reference tables.

This PR allows users to query multiple placements of shards in a round robin fashion. When `citus.task_assignment_policy` is set to `'round-robin'` the planner will use a round robin scheduling feature when multiple shard placements are available.

The primary use-case is spreading the load of reference table queries to all the nodes in the cluster instead of hammering only the first placement of the reference table. Since reference tables share the same path for selecting the shards with single shard queries that have multiple placements (`citus.shard_replication_factor > 1`) this setting also allows users to spread the query load on these shards.

For modifying queries we do not apply a round-robin strategy. This would be negated by an extra reordering step in the executor for such queries where a `first-replica` strategy is enforced.
2018-11-15 15:11:15 +01:00
Marco Slot f383e4f307
Description: Refactor code that handles DDL commands from one file into a module
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.
2018-11-14 13:36:27 +01:00
Murat Tuncer cc401a2616 Create function_utils for pg function call related utilities 2018-11-07 15:29:38 +03:00
Marco Slot d56baefe3d Allow simple DML commands from hot standby 2018-10-06 10:54:44 +02:00
Murat Tuncer 4f8042085c Fix drop schema in mx with partitioned tables
Drop schema command fails in mx mode if there
is a partitioned table with active partitions.

This is due to fact that sql drop trigger receives
all the dropped objects including partitions. When
we call drop table on parent partition, it also drops
the partitions on the mx node. This causes the drop
table command on partitions to fail on mx node because
they are already dropped when the partition parent was
dropped.

With this work we did not require the table to exist on
worker_drop_distributed_table.
2018-10-08 17:01:54 -07:00
velioglu 512d23934f Show router modify,select and real-time queries on MX views 2018-10-02 13:59:38 +03:00
Onder Kalaci abc443d7fa Make sure that shard repair considers replication factor 2018-09-21 15:24:49 +03:00
Onder Kalaci 8520a5b432 worker_append_table_to_shard becomes aware of partitioned tables 2018-09-21 14:40:42 +03:00
Onder Kalaci c1b5a04f6e Allow partitioned tables with replication factor > 1
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.
2018-09-21 14:40:41 +03:00
Murat Tuncer b6930e3db9 Add distributed locking to truncated mx tables
We acquire distributed lock on all mx nodes for truncated
tables before actually doing truncate operation.

This is needed for distributed serialization of the truncate
command without causing a deadlock.
2018-09-21 14:23:19 +03:00
Marco Slot f34ab55389 Fix bug preventing rollback in stored procedure 2018-08-31 20:49:20 +02:00
Onder Kalaci 41d606b575 Use tree walker instad of mutator in relation visibility
This commit uses *_walker instead of *_mutator for performance reasons.
Given that we're only updating a functionId in the tree, the approach
seems fine.
2018-09-18 09:33:01 +03:00
Onder Kalaci a94184fff8 Prevent overflow of memory accesses during deadlock detection
In the distributed deadlock detection design, we concluded that prepared transactions
cannot be part of a distributed deadlock. The idea is that (a) when the transaction
is prepared it already acquires all the locks, so cannot be part of a deadlock
(b) even if some other processes blocked on the prepared transaction,  prepared transactions
 would eventually be committed (or rollbacked) and the system will continue operating.

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

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

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

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

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

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

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

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

- pg_stat_activity columns:
    These two views get all the columns from pg_stat_activity. We're basically joining
    pg_stat_activity with get_all_active_transactions on process_id.
2018-09-10 21:33:27 +03:00
Onder Kalaci 76aa6951c2 Properly send commands to other nodes
We previously implemented OTHER_WORKERS_WITH_METADATA tag. However,
that was wrong. See the related discussion:
     https://github.com/citusdata/citus/issues/2320

Instead, we switched using OTHER_WORKER_NODES and make the command
that we're running optional such that even if the node is not a
metadata node, we won't be in trouble.
2018-09-10 16:01:30 +03:00
Onder Kalaci 5cf8fbe7b6 Add infrastructure to relation if exists 2018-09-07 14:49:36 +03:00
Murat Tuncer 65276311f7 Reflect changed index for constraint scans in PG11 2018-09-07 08:07:01 +03:00
Onder Kalaci 26e308bf2a Support TRUNCATE from the MX worker nodes
This commit enables support for TRUNCATE on both
distributed table and reference tables.

The basic idea is to acquire lock on the relation by sending
the TRUNCATE command to all metedata worker nodes. We only
skip sending the TRUNCATE command to the node that actually
executus the command to prevent a self-distributed-deadlock.
2018-09-03 14:06:31 +03:00
Onder Kalaci 97ba7bf2eb Add the option to skip the node that is executing the node 2018-09-03 14:01:24 +03:00
velioglu bd30e3e908 Add support for writing to reference tables from MX nodes 2018-08-27 18:15:04 +03:00
velioglu 2639149bd8 Enterprise functions about metadata/resource locks 2018-08-27 16:32:20 +03:00
Onder Kalaci b8af8c359b Make sure that modifying CTEs always use the correct execution mode 2018-08-23 14:53:55 +03:00
mehmet furkan şahin ef9f38b68d ApplyLogRedaction noop func is added 2018-08-17 14:48:54 -07:00
Onder Kalaci 974cbf11a5 Hide shard names on MX worker nodes
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.
2018-08-07 14:21:45 +03:00
Onder Kalaci e13da6a343 Add infrastructure to hide shards on MX worker nodes
Add ability to understand whether a table is a
known shard on MX workers. Note that this is only useful
and applicable for hiding shards on MX worker nodes given
that we can have metadata only there.
2018-08-04 09:03:37 +03:00
Nils Dijk 6a15e1c9fc
extract ErrorIfOnConflictNotSupported function for reuse 2018-07-23 12:20:10 +02:00
Jason Petersen 318119910b
Add pg_dist_poolinfo table
For storing nodes' pool host/port overrides.
2018-07-10 09:30:22 -07:00
mehmet furkan şahin 3afa7f425d Topn aggregates are supported 2018-07-10 14:33:42 +03:00
Marco Slot 89870e76ce Add a select_opens_transaction_block GUC 2018-07-08 03:50:39 +02:00
Onder Kalaci 7fb529aab9 Some stylistic improvements in the foreign keys to reference table
changes.
2018-07-05 23:23:34 +03:00
Nils Dijk c1c8c38dc9 create placeholder for policy ddl 2018-07-05 11:07:01 +02:00
mehmet furkan şahin 06217be326 hll aggregate functions are supported natively 2018-07-04 16:41:09 +03:00
mehmet furkan şahin f7b901e3fd CopyShardForeignConstraintCommandList API change for grouped constraints 2018-07-03 17:05:55 +03:00
mehmet furkan şahin 35eac2318d lock referenced reference table metadata is added
For certain operations in enterprise, we need to lock the
referenced reference table shard distribution metadata
2018-07-03 17:05:55 +03:00
Onder Kalaci d83be3a33f Enforce foreign key restrictions inside transaction blocks
When a hash distributed table have a foreign key to a reference
table, there are few restrictions we have to apply in order to
prevent distributed deadlocks or reading wrong results.

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

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

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

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

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

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

   - There is a GUC called citus.enforce_foreign_key_restrictions
     to disable all the checks. We added that GUC since the restrictions
     we apply is sometimes a bit more restrictive than its necessary.
     The user might want to relax those. Similarly, if you don't have
     CASCADEing reference tables, you might consider disabling all the
     checks.
2018-07-03 17:05:55 +03:00
velioglu 6be6911ed9 Create foreign key relation graph and functions to query on it 2018-07-03 17:05:55 +03:00
mehmet furkan şahin 4db72c99f6 Specific DDLs are sequentialized when there is FK
-[x] drop constraint
-[x] drop column
-[x] alter column type
-[x] truncate

are sequentialized if there is a foreign constraint from
a distributed table to a reference table on the affected relations
by the above commands.
2018-07-03 17:05:55 +03:00
mehmet furkan şahin 2c5d59f3a8 create_distributed_table in transaction is fixed 2018-07-03 17:05:01 +03:00
mehmet furkan şahin 2fa4e38841 FK from dist to ref can be added with alter table 2018-07-03 17:05:01 +03:00
Murat Tuncer 3fc7cdfe6d Apply master_stage_protocol refactoring changes 2018-06-28 11:24:57 +03:00
Murat Tuncer 4d35b92016 Add groundwork for citus_stat_statements api 2018-06-27 14:20:03 +03:00
Onder Kalaci 8ccb8b679e Real-time executor marks multi shard relation accesses before opening connections 2018-06-25 18:40:31 +03:00
Onder Kalaci 21038f0d0e Make sure that inter-shard DDL commands are always covers both tables 2018-06-25 18:40:30 +03:00
Onder Kalaci 2f01894589 Track relation accesses using the connection management infrastructure 2018-06-25 18:40:30 +03:00
Onder Kalaci d5472614df Use non-data connection for intermediate results
Make sure that intermediate results use a connection that is
not associated with any placement. That is useful in two ways:
    - More complex queries can be executed with CTEs
    - Safely use the same connections when there is a foreign key
      to reference table from a distributed table, which needs to
      use the same connection for modifications since the reference
      table might cascade to the distributed table.
2018-06-21 13:26:13 +03:00
Onder Kalaci 7762d81cba Move test UDF under test folder 2018-06-21 08:42:44 +03:00
Jason Petersen 7a75c2ed31 Add connparam invalidation trigger creation logic
This needs to live in Community, since we haven't yet added the com-
plication of having divergent upgrade scripts in Enterprise.
2018-06-20 14:13:18 -06:00
velioglu 53b2e81d01 Adds SELECT ... FOR UPDATE support for router plannable queries 2018-06-18 13:55:17 +03:00
Marco Slot 0feb1f2eb1 Do not call CheckRemoteTransactionsHealth from commit handler 2018-06-14 23:33:07 +02:00
Marco Slot 4ab8e87090 Always throw errors on failure on critical connection in router executor 2018-06-14 23:33:07 +02:00
Nils Dijk 73efcb22c4 Extract RoleSpecString and resolve role references 2018-06-14 11:38:42 +02:00
Jason Petersen 57b3f253c5
Add node_conninfo GUC and related logic
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.
2018-06-12 20:23:47 -06:00
mehmet furkan şahin d1a3b20115 foreign_constraint_utils is created 2018-06-07 18:19:24 +03:00
Onder Kalaci 336044f2a8 master_modify_multiple_shards() and TRUNCATE honors multi_shard_modification_mode 2018-06-06 12:29:05 +03:00
Onder Kalaci df44956dc3 Make sure that sequential DDL opens a single connection to each node
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.
2018-06-05 17:52:17 +03:00
Marco Slot fd4ff29f2f Add a debug message with distribution column value 2018-06-05 15:09:17 +03:00
Dimitri Fontaine 8b258cbdb0 Lock reads and writes only to the node being updated in master_update_node
Rather than locking out all the writes in the cluster, the function now only
locks out writes that target shards hosted by the node we're updating.
2018-05-09 15:14:20 +02:00
Murat Tuncer 42a8082721 PG11 compatibility refresh
adds a shim for a changed function api
2018-05-03 13:21:15 -06:00
Onder Kalaci 317dd02a2f Implement single repartitioning on hash distributed tables
* 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
2018-05-02 18:50:55 +03:00
velioglu 32bcd610c1 Support modify queries with multiple tables
With this commit we begin to support modify queries with multiple
tables if these queries are pushdownable.
2018-05-02 16:22:26 +03:00
velioglu d9fa69c031 Refactor query pushdown related logic 2018-05-02 15:03:09 +03:00
velioglu 121ff39b26 Removes large_table_shard_count GUC 2018-04-29 10:34:50 +02:00
mehmet furkan şahin a4153c6ab1 notice handler is implemented 2018-04-27 14:37:01 +03:00
Marco Slot 304b3a41ba Cache the partition column Var 2018-04-26 14:58:16 -06:00
Marco Slot 3d3c19a717
Improve messages for essential connection failures 2018-04-26 12:58:47 -06:00
Murat Tuncer a6fe5ca183 PG11 compatibility update
- 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
2018-04-26 11:29:43 +03:00
Onder Kalaci ee748d9140 Unify extendedOpNode Processing
Before this commit, we had a divergence among
the creation of master/worker extended op nodes.

This commit moves the related parts into a single place
and allows the creation of master/extended op nodes to
share a common data structure.
2018-04-24 11:56:38 +03:00
velioglu 82b2d21b0c Convert broadcast join to reference join
After this commit large_table_shard_count wont be used to
check whether broadcast join, which is renamed as reference
join, can be applied. Reference join can only be applied over
reference tables.
2018-04-13 12:58:14 +03:00
Marco Slot ee132c5ead Prune shards once per relation in subquery pushdown 2018-04-10 20:33:07 +02:00
Burak Yucesoy 0c283fa8a3 Add partitioning support to MX tables
Previously, we prevented creation of partitioned tables on Citus MX.
We decided to not focus on this feature until there is a need. Since
now there are requests for this feature, we are implementing support
for partitioned tables on Citus MX.
2018-04-06 12:47:06 +03:00
velioglu 72dfe4a289 Adds colocation check to local join 2018-04-04 22:49:27 +03:00
velioglu 82a864308a Remove SHARD_STORAGE_RELAY type 2018-03-30 11:45:19 +03:00
velioglu 698d585fb5 Remove broadcast join logic
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.
2018-03-30 11:45:19 +03:00
Murat Tuncer 224b0a8c14 Replace poll with select/poll
Windows does not have poll(), so fall back to select()
2018-03-21 20:05:00 -07:00
Metin Doslu 3b7b64a8b6 Remove skip_jsonb_validation_in_copy GUC 2018-03-13 10:33:27 +02:00
Murat Tuncer 1440caeef2
Fix incorrect limit pushdown when distinct clause is not superset of group by (#2035)
Pushing down limit and order by into workers may produce
wrong output when distinct on() clause has expressions,
aggregates, or window functions.

This checking allows pushing down of limits only if
distinct clause is a superset of group by clause. i.e. it contains all clauses in group by.
2018-03-07 13:24:56 +03:00
Murat Tuncer 76f6883d5d
Add support for window functions that can be pushed down to worker (#2008)
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.
2018-03-01 19:07:07 +03:00
Marco Slot ef5ff7eb12 Add bit_ and bool_ aggregates to AggregateType 2018-02-27 23:48:25 +01:00
Marco Slot c723a1fa32 Add support for bool and bit aggregates 2018-02-27 23:48:25 +01:00
Metin Doslu bcf660475a Add support for modifying CTEs 2018-02-27 15:08:32 +02:00
Onder Kalaci 1c930c96a3 Support non-co-located joins between subqueries
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.
2018-02-26 13:50:37 +02:00
Onder Kalaci 7b57e0562a Add infrastructure for detecting non-colocated subqueries 2018-02-26 13:28:25 +02:00
Onder Kalaci e8aa532a90 Refactor checks for distribution key equality
Change some function names, ensure we stick to Citus'
function order rules etc.
2018-02-26 13:28:24 +02:00
Markus Sintonen 6202e80d06 Implemented jsonb_agg, json_agg, jsonb_object_agg, json_object_agg 2018-02-18 00:19:18 +02:00
velioglu 195ac948d2 Recursively plan subqueries in WHERE clause when FROM recurs 2018-02-13 19:52:12 +03:00
Marco Slot 6051aae56e Handle errors that are discovered during abort 2018-02-12 16:45:02 +01:00
Onder Kalaci 94c5ac6ebb Remove duplicate join restrictions
We use PostgreSQL hooks to accumulate the join restrictions
and PostgreSQL gives us all the join paths it tries while
deciding on the join order. Thus, for queries that have many
joins, this function is likely to remove lots of duplicate join
restrictions. This becomes relevant for Citus on query pushdown
check peformance.
2018-02-12 18:35:05 +02:00
Onder Kalaci c228d8ff3d Refactor equivalance generation related codes
This commit changes the APIs for restriction generation to make future
changes simpler.
2018-02-12 18:35:04 +02:00
Onder Kalaci 2f2d350924 Refactor relation restriction related codes
This commit moves some of the functions to a more relevant
source file.
2018-02-12 18:35:04 +02:00
Marco Slot 6f7c3bd73b Skip JSON validation on coordinator during COPY 2018-02-02 15:33:27 +01:00
Brian Cloutier 15511f6ba1 Dynamically allocate connection metadata in WaitForAllConnections 2018-02-01 10:30:41 -08:00
Brian Cloutier a2ed45e206 Remove variable length arrays
VLAs aren't supported by Visual Studio.

- Remove all existing instances of VLAs.
- Add a flag, -Werror=vla, which makes gcc refuse to compile if we add
  VLAs in the future.
2018-02-01 10:30:41 -08:00
Brian Cloutier 76d1edc3fd
Don't rely on gcc-specific features (#1963)
* Don't use expressions inside compound statements
* Don't depend on __builtin_constant_p
* Remove reliance on S_ISLNK
* Replace use of __func__: older mcvs doesn't support this builtin
2018-01-23 17:03:29 -08:00
Marco Slot 3fd65cb91b Do not raise errors in the real-time executor (#1903) 2018-01-01 22:26:31 -05:00
Marco Slot 09c09f650f Recursively plan set operations when leaf nodes recur 2017-12-26 13:46:55 +02:00
mehmet furkan şahin fd546cf322 Intermediate result size limitation
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.
2017-12-21 14:26:56 +03:00
Onder Kalaci 0d5a4b9c72 Recursively plan subqueries that are not safe to pushdown
With this commit, Citus recursively plans subqueries that
are not safe to pushdown, in other words, requires a merge
step.

The algorithm is simple: Recursively traverse the query from bottom
up (i.e., bottom meaning the leaf queries). On each level, check
whether the query is safe to pushdown (or a single repartition
subquery). If the answer is yes, do not touch that subquery. If the
answer is no, plan the subquery seperately (i.e., create a subPlan
for it) and replace the subquery with a call to
`read_intermediate_results(planId, subPlanId)`. During the the
execution, run the subPlans first, and make them avaliable to the
next query executions.

Some of the queries hat this change allows us:

   * Subqueries with LIMIT
   * Subqueries with GROUP BY/DISTINCT on non-partition keys
   * Subqueries involving re-partition joins, router queries
   * Mixed usage of subqueries and CTEs (i.e., use CTEs in
     subqueries as well). Nested subqueries as long as we
     support the subquery inside the nested subquery.
   * Subqueries with local tables (i.e., those subqueries
     has the limitation that they have to be leaf subqueries)

   * VIEWs on the distributed tables just works (i.e., the
     limitations mentioned below still applies to views)

Some of the queries that is still NOT supported:

  * Corrolated subqueries that are not safe to pushdown
  * Window function on non-partition keys
  * Recursively planned subqueries or CTEs on the outer
    side of an outer join
  * Only recursively planned subqueries and CTEs in the FROM
    (i.e., not any distributed tables in the FROM) and subqueries
    in WHERE clause
  * Subquery joins that are not on the partition columns (i.e., each
    subquery is individually joined on partition keys but not the upper
    level subquery.)
  * Any limitation that logical planner applies such as aggregate
    distincts (except for count) when GROUP BY is on non-partition key,
    or array_agg with ORDER BY
2017-12-21 08:37:40 +02:00
Onder Kalaci e12ea914b9 Refactor ErrorIfQueryNotSupported to defer errors 2017-12-20 09:03:49 +02:00
Onder Kalaci 71ce42b936 Refactor RecursivelyPlanSubqueriesAndCTEs() to make it ready
to work with subqueries
2017-12-20 09:03:47 +02:00
Marco Slot 7dab078e67 Set cost estimates for read_intermediate_result 2017-12-18 16:23:44 +01:00
Marco Slot a64f0060ba Reduce the frequency of FinishConnectionIO calls during COPY (#1864) 2017-12-14 13:21:59 -05:00
Marco Slot 2e2b4e81fa Add support for CTEs in distributed queries 2017-12-14 09:32:55 +01:00
Marco Slot cbbd418af2 Add citus.copy_format OIDs to metadata cache 2017-12-14 09:32:55 +01:00
Marco Slot 66f9f1d6cd Make some intermediate results functions public 2017-12-14 09:32:55 +01:00
Marco Slot 36ee21c323 Make CanUseBinaryCopyFormatForType public 2017-12-14 09:32:55 +01:00
Marco Slot 7d1191954d Add DistributedSubPlan node 2017-12-14 09:32:55 +01:00
mehmet furkan şahin 3c941aedf1 adds citus.enable_repartition_joins GUC
The new GUC allows Citus to switch between task executors
when necessary
2017-12-11 09:36:37 +03:00
Marco Slot 60a1e31671 Allow queries with local tables in NeedsDistributedPlanning 2017-12-07 16:20:23 +01:00
Marco Slot f8550b8c85 Fix issues with read_intermediate_result signature 2017-12-07 13:47:56 +01:00
Marco Slot d8fea4efb8 Revert "Allow queries with local tables in NeedsDistributedPlanning"
This reverts commit d2bac081e8.
2017-12-07 11:19:11 +01:00
Marco Slot d2bac081e8 Allow queries with local tables in NeedsDistributedPlanning 2017-12-07 11:02:16 +01:00
Marco Slot 7279d42849 Treat read_intermediate_result as recurring tuples 2017-12-04 14:50:11 +01:00
Marco Slot 4cdadfcab6 Add intermediate results infrastructure 2017-12-04 14:50:11 +01:00
Marco Slot bfcc76df69 Make several COPY-related functions public 2017-12-04 13:12:03 +01:00
Marco Slot 73989b07eb Refactor query execution functions 2017-12-04 13:12:03 +01:00
Murat Tuncer 2d66bf5f16
Fix hard coded formatting strings for 64 bit numbers (#1831)
Postgres provides OS agnosting formatting macros for
formatting 64 bit numbers. Replaced %ld %lu with
INT64_FORMAT and UINT64_FORMAT respectively.

Also found some incorrect usages of formatting
flags and fixed them.
2017-12-04 14:11:06 +03:00
Marco Slot d6dd0b3a81 Send BEGIN in the real-time executor when in a transaction 2017-11-30 12:59:09 +01:00
Marco Slot 3a4d5f8182 Remove filter checks on leaf queries 2017-11-30 12:25:14 +01:00
Marco Slot 3f03cb6a6a Support UNION with joins in the subqueries 2017-11-30 10:37:56 +01:00
Marco Slot a9933deac6 Make real time executor work in transactions 2017-11-30 09:59:32 +03:00
Onder Kalaci 05fb0dd020 Add infrastructure for filtering restriction contexts based on the input query
In subquery pushdown, we first ensure that each relation is joined with at least
on another relation on the partition keys. That's fine given that the decision
is binary: pushdown the query at all or not.

With recursive planning, we'd want to check whether any specific part
of the query can be pushded down or not. Thus, we need the ability to
understand which part(s) of the subquery is safe to pushdown. This commit
adds the infrastructure for doing that.
2017-11-28 09:58:21 +02:00
Onder Kalaci 16421f089f Register citus custom scan nodes 2017-11-23 11:38:33 +02:00
Onder Kalaci 83c1143505 Refactor custom scan related codes
In this commit, we don't change any codes, only create a new
file and move the related functions and types there.
2017-11-23 11:38:12 +02:00
Marco Slot 8486f76e15 Auto-recover 2PC transactions 2017-11-22 11:26:58 +01:00
Marco Slot 6ba3f42d23 Rename MultiPlan to DistributedPlan 2017-11-22 09:36:24 +01:00
Marco Slot 0ad39b36fe Treat immutable table functions and constant subqueries as reference tables 2017-11-21 14:15:22 +01:00
Brian Cloutier d267e0f9fa EXEC_BACKEND: don't put pointers to shared hashes into shared memory
Store pointers to shared hashes in process-local variables. Previously
pointers to shared hashes were put into shared memory. This causes
problems on EXEC_BACKEND because everybody calls execve and receives a
brand new address space; the shared hash will be in a different place
for every backend. (normally we call fork, which gives you a copy of the
address space, so these pointers remain constant)
2017-11-20 15:29:51 -08:00
Brian Cloutier 30a2365d81 Rename CreateDirectory to CitusCreateDirectory 2017-11-20 14:38:26 -08:00
Brian Cloutier aa2ab023a2 Rename RemoveDirectory -> CitusRemoveDirectory 2017-11-20 14:21:52 -08:00
Marco Slot 2410c2e450 Rewrite recover_prepared_transactions to be fast, non-blocking 2017-11-20 11:27:40 +01:00
Marco Slot d3b634b301 Allow generating placement IDs without using the sequence 2017-11-15 10:12:06 +01:00
Marco Slot c24a0875a5 Allow generating shard IDs without using the sequence 2017-11-15 10:12:05 +01:00
velioglu be28ba8e70 Add stub UDF to run pg_upgrade flawlessly 2017-11-13 16:14:45 +02:00
Marco Slot f71728f634 Add GUC for specifying sslmode in connections to workers 2017-11-08 14:15:58 +01:00
Brian Cloutier 7be1545843 Support implicit casts during INSERT/SELECT
It's possible to build INSERT SELECT queries which include implicit
casts, currently we attempt to support these by adding explicit casts to
the SELECT query, but this sometimes crashes because we don't update all
nodes with the new types. (SortClauses, for instance)

This commit removes those explicit casts and passes an unmodified SELECT
query to the COPY executor (how we implement INSERT SELECT under the
scenes). In lieu of those cases, COPY has been given some extra logic to
inspect queries, notice that the types don't line up with the table it's
supposed to be inserting into, and "manually" casting every tuple before
sending them to workers.
2017-11-03 22:27:15 -07:00
Marco Slot 6883a09cdd Allow distributed partitioned table creation in Cloud 2017-11-03 10:09:18 +01:00
Hadi Moshayedi 9bfbbf8a04 Make reports hostname configurable and enable stats collection in tests.
This patch adds --with-reports-host configure option, which sets the
REPORTS_BASE_URL constant. The default is reports.citusdata.com.

It also enables stats collection in tests.
2017-10-31 21:51:43 -04:00
Hadi Moshayedi 78a2cd9052 Check for Citus updates.
Sends a request to /v1/releases/latest?flavor=$CITUS_EDITION once a day,
which returns a response similar to {"version": "7.1.0", "major": 7,
"minor": 1, "patch": 0}. Then compares it with current Citus version,
and if the latest release is newer, logs a LOG message.
2017-10-31 21:51:43 -04:00
Hadi Moshayedi 34f3ec0961 Call FlushDistTableCache() before stats collection. 2017-10-31 21:51:43 -04:00
Hadi Moshayedi 97d544b75c Follow the patterns used in Deadlock Detection in Stats Collection.
This includes:

(1) Wrap everything inside a StartTransactionCommand()/CommitTransactionCommand().
This is so we can access the database. This also switches to a new memory context
and releases it, so we don't have to do our own memory management.

(2) LockCitusExtension() so the extension cannot be dropped or created concurrently.

(3) Check CitusHasBeenLoaded() && CheckCitusVersion() before doing any work.

(4) Do not PG_TRY() inside a loop.
2017-10-31 21:51:43 -04:00
Furkan Sahin 2b39c52f0b Replica identity on create_distributed_table
By this commit, citus minds the replica identity of the table when
we distribute the table. So the shards of the distributed table
have the same replica identity with the local table.
2017-10-31 13:08:36 +03:00