Commit Graph

139 Commits (29d45bd1b912c14a304235468d72bc8a2e4e9480)

Author SHA1 Message Date
SaitTalhaNisanci 29d45bd1b9
Do not assign InvalidOid for local execution while extracting parameters (#3131)
* do not assign InvalidOid for local execution while extracting parameters

* rename functions

* rename parameter and replace function
2019-10-28 14:28:22 +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
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
Nils Dijk 2879689441
Distribute Types to worker nodes (#2893)
DESCRIPTION: Distribute Types to worker nodes

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

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

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

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

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

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

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

Lets assume the following transaction:

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

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

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

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

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

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

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

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

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

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

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

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

All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables.
2019-09-13 17:46:07 +02:00
Onder Kalaci 0b0c779c77 Introduce the concept of Local Execution
/*
 * local_executor.c
 *
 * The scope of the local execution is locally executing the queries on the
 * shards. In other words, local execution does not deal with any local tables
 * that are not shards on the node that the query is being executed. In that sense,
 * the local executor is only triggered if the node has both the metadata and the
 * shards (e.g., only Citus MX worker nodes).
 *
 * The goal of the local execution is to skip the unnecessary network round-trip
 * happening on the node itself. Instead, identify the locally executable tasks and
 * simply call PostgreSQL's planner and executor.
 *
 * The local executor is an extension of the adaptive executor. So, the executor uses
 * adaptive executor's custom scan nodes.
 *
 * One thing to note that Citus MX is only supported with replication factor = 1, so
 * keep that in mind while continuing the comments below.
 *
 * On the high level, there are 3 slightly different ways of utilizing local execution:
 *
 * (1) Execution of local single shard queries of a distributed table
 *
 *      This is the simplest case. The executor kicks at the start of the adaptive
 *      executor, and since the query is only a single task the execution finishes
 *      without going to the network at all.
 *
 *      Even if there is a transaction block (or recursively planned CTEs), as long
 *      as the queries hit the shards on the same, the local execution will kick in.
 *
 * (2) Execution of local single queries and remote multi-shard queries
 *
 *      The rule is simple. If a transaction block starts with a local query execution,
 *      all the other queries in the same transaction block that touch any local shard
 *      have to use the local execution. Although this sounds restrictive, we prefer to
 *      implement in this way, otherwise we'd end-up with as complex scenarious as we
 *      have in the connection managements due to foreign keys.
 *
 *      See the following example:
 *      BEGIN;
 *          -- assume that the query is executed locally
 *          SELECT count(*) FROM test WHERE key = 1;
 *
 *          -- at this point, all the shards that reside on the
 *          -- node is executed locally one-by-one. After those finishes
 *          -- the remaining tasks are handled by adaptive executor
 *          SELECT count(*) FROM test;
 *
 *
 * (3) Modifications of reference tables
 *
 *		Modifications to reference tables have to be executed on all nodes. So, after the
 *		local execution, the adaptive executor keeps continuing the execution on the other
 *		nodes.
 *
 *		Note that for read-only queries, after the local execution, there is no need to
 *		kick in adaptive executor.
 *
 *  There are also few limitations/trade-offs that is worth mentioning. First, the
 *  local execution on multiple shards might be slow because the execution has to
 *  happen one task at a time (e.g., no parallelism). Second, if a transaction
 *  block/CTE starts with a multi-shard command, we do not use local query execution
 *  since local execution is sequential. Basically, we do not want to lose parallelism
 *  across local tasks by switching to local execution. Third, the local execution
 *  currently only supports queries. In other words, any utility commands like TRUNCATE,
 *  fails if the command is executed after a local execution inside a transaction block.
 *  Forth, the local execution cannot be mixed with the executors other than adaptive,
 *  namely task-tracker, real-time and router executors. Finally, related with the
 *  previous item, COPY command cannot be mixed with local execution in a transaction.
 *  The implication of that any part of INSERT..SELECT via coordinator cannot happen
 *  via the local execution.
 */
2019-09-12 11:51:25 +02:00
Philip Dubé 018ad1c58e pg12: version_compat.h, tuples, oids, misc 2019-08-22 18:57:23 +00:00
Hadi Moshayedi 009d8b7401 Some cleanup 2019-08-12 15:38:52 -07: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
Ö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
Philip Dubé 5c62f9935a Router planner: reject SELECT FOR UPDATE ctes 2019-06-26 10:32:01 +02:00
Onder Kalaci ad93d6feea Change the order of placement access added to the list
This is to make sure that the error messages related to foreign keys
to reference tables shows the exact placement access name instead of
SELECT.
2019-06-23 11:32:58 +02: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
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
Marco Slot bb3a96eacb Cache a configurable number of connections at xact end 2019-05-29 13:24:31 +02:00
Onder Kalaci 495b6e9b62 Refactor Parallel Relation Access Recording
Instead of scattering the code around, we move all the
logic into a single function.

This will help supporting foreign keys to reference tables
in the unified executor with a single line of change, just
calling this function.
2019-05-02 18:12:33 +03:00
Hadi Moshayedi 32ecb6884c Test ROLLBACK TO SAVEPOINT with multi-shard CTE failures 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
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
Marco Slot f2abf2b8e5 Functions are treated as transaction blocks 2019-03-15 16:34:08 -06:00
Hadi Moshayedi f4d3b94e22
Fix some of the casts for groupId (#2609)
A small change which partially addresses #2608.
2019-03-05 12:06:44 -08:00
Marco Slot 1b1c6374f7
Execute CREATE INDEX CONCURRENTLY concurrently 2018-12-21 14:02:59 -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 aff37cf1bc Control multi-shard modify locks with enable_deadlock_prevention 2018-11-28 02:59:50 +01:00
Onder Kalaci 6e05921736 Processes that are blocked on advisory locks show up in wait edges
Assign the distributed transaction id before trying to acquire the
executor advisory locks. This is useful to show this backend in citus
lock graphs (e.g., dump_global_wait_edges() and citus_lock_waits).
2018-10-24 13:32:13 +03:00
velioglu 512d23934f Show router modify,select and real-time queries on MX views 2018-10-02 13:59:38 +03:00
Marco Slot f34ab55389 Fix bug preventing rollback in stored procedure 2018-08-31 20:49:20 +02:00
velioglu bd30e3e908 Add support for writing to reference tables from MX nodes 2018-08-27 18:15:04 +03:00
Onder Kalaci b8af8c359b Make sure that modifying CTEs always use the correct execution mode 2018-08-23 14:53:55 +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 2890154420 Make sure that TRUNCATE always opens a DDL access 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
velioglu 53b2e81d01 Adds SELECT ... FOR UPDATE support for router plannable queries 2018-06-18 13:55:17 +03:00
Marco Slot 0bbe778760 Rename failOnError to alwaysThrowErrorOnFailure 2018-06-14 23:37:47 +02:00
Marco Slot 4ab8e87090 Always throw errors on failure on critical connection in router executor 2018-06-14 23:33:07 +02: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
Marco Slot 5f5f7b4fe0 Throw an error if placements cannot be found in router executor 2018-05-08 22:39:18 -04:00
Marco Slot 90cdfff602 Implement recursive planning for DML statements 2018-05-03 14:42:28 +02: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
mehmet furkan şahin a4153c6ab1 notice handler is implemented 2018-04-27 14:37:01 +03: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 7dc9589b56 Handle failures during I/O
This commit checks the connection status right after any IO happens
on the socket.

This is necessary since before this commit we didn't pass any information
to the higher level functions whether we're done with the connection
(e.g., no IO required anymore) or an errors happened during the IO.
2018-03-02 08:33:53 +02:00
Metin Doslu bcf660475a Add support for modifying CTEs 2018-02-27 15:08:32 +02:00