Commit Graph

165 Commits (85324f3accbfeee3375293c6bbf6c332778622c4)

Author SHA1 Message Date
SaitTalhaNisanci 42cfc4c0e9 apply review items
log shard id in local copy and add more comments
2020-03-18 09:33:55 +03:00
SaitTalhaNisanci 1df9601e13 not use local copy if current transaction is connected to local group
If current transaction is connected to local group we should not use
local copy, because we might not see some of the changes that are made
over the connection to the local group.
2020-03-18 09:28:59 +03:00
SaitTalhaNisanci f9c4431885 add the support to execute copy locally
A copy will be executed locally if
- Local execution is enabled and current transaction accessed a local placement
- Local execution is enabled and we are inside a transaction block.

So even if local execution is enabled but we are not in a transaction block, the copy will not be run locally.

This will not run locally:
```
COPY distributed_table FROM STDIN;
....
```

This will run locally:
```
SET citus.enable_local_execution to 'on';
BEGIN;
COPY distributed_table FROM STDIN;
COMMIT;
....
```
.
There are 3 ways to do a copy in postgres programmatically:
- from a file
- from a program
- from a callback function

I have chosen to implement it with a callback function, which means that we write the rows of copy from a callback function to the output buffer, which is used to insert tuples into the actual table.

For each shard id, we have a buffer that keeps the current rows to be written, we perform the actual copy operation either when:
- copy buffer for the given shard id reaches to a threshold, which is currently 512KB
- we reach to the end of the copy

The buffer size is debatable(512KB). At a given time, we might allocate (local placement * buffer size) memory at most.

The local copy uses the same copy format as remote copy, which means that we serialize the data in the same format as remote copy and send it locally.

There was also the option to use ExecSimpleRelationInsert to insert
slots one by one, which would avoid the extra
serialization/deserialization but doing some benchmarks it seems that
using buffers are significantly better in terms of the performance.

You can see this comment for more details: https://github.com/citusdata/citus/pull/3557#discussion_r389499054
2020-03-18 09:28:59 +03:00
Philip Dubé 7cdfa1daab Rename LookupCitusTableCacheEntry to GetCitusTableCacheEntry, LookupLookupCitusTableCacheEntry back to LookupCitusTableCacheEntry 2020-03-08 14:08:23 +00:00
Philip Dubé a7cca1bcde Rename DistTableCacheEntry to CitusTableCacheEntry 2020-03-07 14:08:03 +00:00
Philip Dubé b514ab0f55 Fix typos, rename isDistributedRelation to isCitusRelation 2020-03-06 19:20:34 +00:00
Philip Dubé bec58000d6 Given IsDistributedTableRTE, there's ambiguity in what DistributedTable means
Elsewhere we used DistributedTable to include reference tables
Marco suggested we use CitusTable for distributed & reference tables

So renaming:
- IsDistributedTable -> IsCitusTable
- IsDistributedTableViaCatalog -> IsCitusTableViaCatalog
- DistributedTableCacheEntry -> CitusTableCacheEntry
- DistributedTableList -> CitusTableList
- isDistributedTable -> isCitusTable
- InsertSelectIntoDistributedTable -> InsertSelectIntoCitusTable
- ExtractFirstDistributedTableId -> ExtractFirstCitusTableId
2020-03-06 18:57:55 +00:00
Philip Dubé 34f241af16 Fix create_distributed_table on a table using GENERATED ALWAYS AS
If the generated column does not come at the end of the column list,
columnNameList doesn't line up with the column indexes. Seek past

CREATE TABLE test_table (
    test_id int PRIMARY KEY,
    gen_n int GENERATED ALWAYS AS (1) STORED,
    created_at TIMESTAMPTZ NOT NULL DEFAULT now()
);
SELECT create_distributed_table('test_table', 'test_id');

Would raise ERROR: cannot cast 23 to 1184
2020-02-28 09:34:26 -08:00
Jelte Fennema 8de8b62669 Convert unsafe APIs to safe ones 2020-02-25 15:39:27 +01:00
Marco Slot 038e5999cb Implement direct COPY table TO stdout 2020-02-17 15:15:10 +01:00
Önder Kalacı ef7d1ea91d
Locally execute queries that don't need any data access (#3410)
* Update shardPlacement->nodeId to uint

As the source of the shardPlacement->nodeId is always workerNode->nodeId,
and that is uint32.

We had this hack because of: 0ea4e52df5 (r266421409)

And, that is gone with: 90056f7d3c (diff-c532177d74c72d3f0e7cd10e448ab3c6L1123)

So, we're safe to do it now.

* Relax the restrictions on using the local execution

Previously, whenever any local execution happens, we disabled further
commands to do any remote queries. The basic motivation for doing that
is to prevent any accesses in the same transaction block to access the
same placements over multiple sessions: one is local session the other
is remote session to the same placement.

However, the current implementation does not distinguish local accesses
being to a placement or not. For example, we could have local accesses
that only touches intermediate results. In that case, we should not
implement the same restrictions as they become useless.

So, this is a pre-requisite for executing the intermediate result only
queries locally.

* Update the error messages

As the underlying implementation has changed, reflect it in the error
messages.

* Keep track of connections to local node

With this commit, we're adding infrastructure to track if any connection
to the same local host is done or not.

The main motivation for doing this is that we've previously were more
conservative about not choosing local execution. Simply, we disallowed
local execution if any connection to any remote node is done. However,
if we want to use local execution for intermediate result only queries,
this'd be annoying because we expect all queries to touch remote node
before the final query.

Note that this approach is still limiting in Citus MX case, but for now
we can ignore that.

* Formalize the concept of Local Node

Also some minor refactoring while creating the dummy placement

* Write intermediate results locally when the results are only needed locally

Before this commit, Citus used to always broadcast all the intermediate
results to remote nodes. However, it is possible to skip pushing
the results to remote nodes always.

There are two notable cases for doing that:

   (a) When the query consists of only intermediate results
   (b) When the query is a zero shard query

In both of the above cases, we don't need to access any data on the shards. So,
it is a valuable optimization to skip pushing the results to remote nodes.

The pattern mentioned in (a) is actually a common patterns that Citus users
use in practice. For example, if you have the following query:

WITH cte_1 AS (...), cte_2 AS (....), ... cte_n (...)
SELECT ... FROM cte_1 JOIN cte_2 .... JOIN cte_n ...;

The final query could be operating only on intermediate results. With this patch,
the intermediate results of the ctes are not unnecessarily pushed to remote
nodes.

* Add specific regression tests

As there are edge cases in Citus MX and with round-robin policy,
use the same queries on those cases as well.

* Fix failure tests

By forcing not to use local execution for intermediate results since
all the tests expects the results to be pushed remotely.

* Fix flaky test

* Apply code-review feedback

Mostly style changes

* Limit the max value of pg_dist_node_seq to reserve for internal use
2020-01-23 18:28:34 +01:00
Hadi Moshayedi b4e5f4b10a Implement INSERT ... SELECT with repartitioning 2020-01-16 23:24:52 -08:00
Marco Slot 90056f7d3c Remove copy from worker for append-partitioned table 2020-01-13 23:03:40 -08:00
Philip Dubé 4b5d6c3ebe Rename RelayFileState to ShardState
Replace FILE_ prefix with SHARD_STATE_
2020-01-12 05:57:53 +00:00
Hadi Moshayedi ef487e0792 Implement fetch_intermediate_results 2019-12-18 10:46:35 -08:00
SaitTalhaNisanci 2829c601dd
replace Begin words in coordinated transactions with use (#3293) 2019-12-16 10:40:31 +03:00
Jelte Fennema 1d8dde232f
Automatically convert useless declarations using regex replace (#3181)
* Add declaration removal to CI

* Convert declarations
2019-11-21 13:47:29 +01:00
SaitTalhaNisanci b9b7fd7660
add IsLoggableLevel utility function (#3149)
* add IsLoggableLevel utility function

* add function comment for IsLoggableLevel

* put ApplyLogRedaction to logutils
2019-11-15 14:59:13 +03:00
Jelte Fennema 1b2c438e69
Rename variables to not shadow globals in RHEL6 (#3194)
Fixes #2839
2019-11-15 12:12:24 +01:00
Philip Dubé 48552bfffe Call DestReceiver rDestroy before it goes out of scope
CitusCopyDestReceiverDestroy: call hash_destroy on shardStateHash & connectionStateHash
2019-11-12 15:03:07 +00:00
Önder Kalacı 960cd02c67
Remove real time router executors (#3142)
* Remove unused executor codes

All of the codes of real-time executor. Some functions
in router executor still remains there because there
are common functions. We'll move them to accurate places
in the follow-up commits.

* Move GUCs to transaction mngnt and remove unused struct

* Update test output

* Get rid of references of real-time executor from code

* Warn if real-time executor is picked

* Remove lots of unused connection codes

* Removed unused code for connection restrictions

Real-time and router executors cannot handle re-using of the existing
connections within a transaction block.

Adaptive executor and COPY can re-use the connections. So, there is no
reason to keep the code around for applying the restrictions in the
placement connection logic.
2019-11-05 12:48:10 +01:00
SaitTalhaNisanci 94a7e6475c
Remove copyright years (#2918)
* Update year as 2012-2019

* Remove copyright years
2019-10-15 17:44:30 +03:00
Marco Slot e269d990c9 Cast the distribution argument value when possible 2019-09-24 17:31:09 +00: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é 41dca121e2 Support GENERATE ALWAYS AS STORED 2019-09-04 14:50:17 +00:00
Philip Dubé 693d4695d7 Create a test 'pg12' for pg12 features & error on unsupported new features
Unsupported new features: COPY FROM WHERE, GENERATED ALWAYS AS, non-heap table access methods
2019-08-22 19:30:56 +00:00
Philip Dubé 018ad1c58e pg12: version_compat.h, tuples, oids, misc 2019-08-22 18:57:23 +00: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
Ö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
Hadi Moshayedi 4bbae02778 Make COPY compatible with unified executor. 2019-06-20 19:53:40 +02:00
Philip Dubé 4bfcf5b665 Enable Werror for all warnings
Changes to ruleutils match changes made upstream to silence gcc fallthrough warnings
2019-06-18 14:43:54 -07:00
Hadi Moshayedi dee5bc31b4 Refactor ShardIdForTuple() to a separate function. 2019-06-02 09:48:15 -07:00
Hadi Moshayedi 8ae47e1244 Fix comments for RemoteFileDestReceiverStartup and CitusCopyDestReceiverStartup 2019-05-21 09:03:22 -07:00
Hadi Moshayedi b5c0ca45f1 Remove stopOnFailure flag from EndRemoteCopy() 2019-05-11 06:18:34 -07:00
Hadi Moshayedi 32ecb6884c Test ROLLBACK TO SAVEPOINT with multi-shard CTE failures 2019-05-01 09:33:43 -07:00
Hadi Moshayedi aafd22dffa Fix savepoint rollback for INSERT INTO ... SELECT. 2019-05-01 09:33:43 -07: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
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 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 6aa5592e52 Add user ID suffix to intermediate files in re-partition jobs 2018-11-23 08:36:11 +01:00
Marco Slot caf402d506 COPY to a task file no longer switches to superuser 2018-11-22 18:15:33 +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
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
velioglu bd30e3e908 Add support for writing to reference tables from MX nodes 2018-08-27 18:15:04 +03:00
mehmet furkan şahin ef9f38b68d ApplyLogRedaction noop func is added 2018-08-17 14:48:54 -07: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
Onder Kalaci 2f01894589 Track relation accesses using the connection management infrastructure 2018-06-25 18:40:30 +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
Brian Cloutier 42ddfa176d Fix crash on Windows where there is no detail 2018-04-13 12:54:22 -07:00
Metin Doslu 3b7b64a8b6 Remove skip_jsonb_validation_in_copy GUC 2018-03-13 10:33:27 +02:00
Marco Slot 6f7c3bd73b Skip JSON validation on coordinator during COPY 2018-02-02 15:33:27 +01:00
Marco Slot 36ee21c323 Make CanUseBinaryCopyFormatForType public 2017-12-14 09:32:55 +01:00
Marco Slot bfcc76df69 Make several COPY-related functions public 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
Hadi Moshayedi ff706cf556 Test that COPY blocks UPDATE/DELETE/INSERT...SELECT when rep factor 2. 2017-11-30 14:52:29 -05:00
Marco Slot acbc0fe0de Use RowExclusiveLock shard resource lock in COPY 2017-11-30 09:15:45 -05:00
Marco Slot a9933deac6 Make real time executor work in transactions 2017-11-30 09:59:32 +03: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
Jason Petersen 6c9b19a954
Add version-compat header
For polyfill macros, etc.
2017-09-25 17:20:23 -07:00
Jason Petersen fbeaa2f9d0
Remove direct access to tupleDesc->attrs
A level of indirection was removed from this field for PostgreSQL 11.
By using the handy provided macro, we can be version agnostic.
2017-09-25 17:20:23 -07:00
Onder Kalaci 4782f9f98a Properly copy and trim the error messages that come from pg_conn
When a NULL connection is provided to PQerrorMessage(), the
returned error message is a static text. Modifying that static
text, which doesn't necessarly be in a writeable memory, is
dangreous and might cause a segfault.
2017-09-22 19:43:09 +03:00
Marco Slot cbe16169b4 Free per-tuple COPY memory in INSERT...SELECT 2017-09-12 15:35:53 -07:00
Marco Slot cf375d6a66 Consider dropped columns that precede the partition column in COPY 2017-08-22 13:02:35 +02:00
Marco Slot 4614814de1 Enable 2PC for INSERT...SELECT via coordinator 2017-08-15 13:44:20 +02:00
Marco Slot 9232823070 Abort on failure on master connection during copy from worker 2017-08-15 13:44:20 +02:00
Burak Yucesoy 2eee556738 Add distributed partitioned table support for COPY
For partitioned tables, PostgreSQL opens partition and its partitions
in BeginCopyFrom and it expects its caller to close those relations.
However, we do not have quick access to opened relations and performing
special operations for partitioned tables isn't necessary in coordinator
node. Therefore before calling BeginCopyFrom, we change relkind of those
partitioned tables to RELKIND_RELATION. This prevents PostgreSQL to open
its partitions as well.
2017-08-09 10:01:35 +03:00
Marco Slot d3e9746236 Avoid connections that accessed non-colocated placements in multi-shard commands 2017-08-08 18:32:34 +02:00
Andres Freund 90a2d13a64 Move multi_copy.c to interrupt aware libpq wrappers. 2017-07-04 14:46:03 -07:00
Andres Freund 3dedeadb5e Fix memory leak in RemoteFinalizedShardPlacementList(). 2017-07-04 12:38:52 -07:00
Andres Freund b96ba9b490 Fix code only enabled for 9.5.
There's still supporting wrappers used, a subsequent commit will
remove those.

This also removes the already unused tuplecount_t define.
2017-06-26 08:46:32 -07:00
Jason Petersen 2204da19f0 Support PostgreSQL 10 (#1379)
Adds support for PostgreSQL 10 by copying in the requisite ruleutils
and updating all API usages to conform with changes in PostgreSQL 10.
Most changes are fairly minor but they are numerous. One particular
obstacle was the change in \d behavior in PostgreSQL 10's psql; I had
to add SQL implementations (views, mostly) to mimic the pre-10 output.
2017-06-26 02:35:46 -06:00
Marco Slot a6f42e4948 Clarify error message when copying NULL value into table 2017-06-22 15:48:24 +02:00
Marco Slot 2f8ac82660 Execute INSERT..SELECT via coordinator if it cannot be pushed down
Add a second implementation of INSERT INTO distributed_table SELECT ... that is used if
the query cannot be pushed down. The basic idea is to execute the SELECT query separately
and pass the results into the distributed table using a CopyDestReceiver, which is also
used for COPY and create_distributed_table. When planning the SELECT, we go through
planner hooks again, which means the SELECT can also be a distributed query.

EXPLAIN is supported, but EXPLAIN ANALYZE is not because preventing double execution was
a lot more complicated in this case.
2017-06-22 15:46:30 +02:00
Marco Slot 2cd358ad1a Support quoted column-names in COPY logic 2017-06-22 15:45:57 +02:00
Marco Slot 70abfd29d2 Allow COPY after a multi-shard command
This change removes the XactModificationLevel check at the start of COPY
that was made redundant by consistently using GetPlacementConnection.
2017-06-09 13:54:58 +02:00
Andres Freund d399f395f7 Faster shard pruning.
So far citus used postgres' predicate proofing logic for shard
pruning, except for INSERT and COPY which were already optimized for
speed.  That turns out to be too slow:
* Shard pruning for SELECTs is currently O(#shards), because
  PruneShardList calls predicate_refuted_by() for every
  shard. Obviously using an O(N) type algorithm for general pruning
  isn't good.
* predicate_refuted_by() is quite expensive on its own right. That's
  primarily because it's optimized for doing a single refutation
  proof, rather than performing the same proof over and over.
* predicate_refuted_by() does not keep persistent state (see 2.) for
  function calls, which means that a lot of syscache lookups will be
  performed. That's particularly bad if the partitioning key is a
  composite key, because without a persistent FunctionCallInfo
  record_cmp() has to repeatedly look-up the type definition of the
  composite key. That's quite expensive.

Thus replace this with custom-code that works in two phases:
1) Search restrictions for constraints that can be pruned upon
2) Use those restrictions to search for matching shards in the most
   efficient manner available:
   a) Binary search / Hash Lookup in case of hash partitioned tables
   b) Binary search for equal clauses in case of range or append
      tables without overlapping shards.
   c) Binary search for inequality clauses, searching for both lower
      and upper boundaries, again in case of range or append
      tables without overlapping shards.
   d) exhaustive search testing each ShardInterval

My measurements suggest that we are considerably, often orders of
magnitude, faster than the previous solution, even if we have to fall
back to exhaustive pruning.
2017-04-28 14:40:41 -07:00
Jason Petersen 42ee7c05f5
Refactor FindShardInterval to use cacheEntry
All callers fetch a cache entry and extract/compute arguments for the
eventual FindShardInterval call, so it makes more sense to refactor
into that function itself; this solves the use-after-free bug, too.
2017-04-27 13:32:36 -06:00
velioglu 1fb11c738f Check binary output function of type. 2017-04-10 16:28:09 +03:00
Marco Slot db98c28354 Address review feedback in COPY refactoring 2017-02-28 17:39:45 +01:00
Marco Slot d74fb764b1 Use CitusCopyDestReceiver for regular COPY 2017-02-28 17:24:45 +01:00
Marco Slot d11eca7d4a Load data into distributed table on creation 2017-02-28 17:24:45 +01:00
Marco Slot bf3541cb24 Add CitusCopyDestReceiver infrastructure 2017-02-28 17:24:45 +01:00
Brian Cloutier 1173f3f225 Refactor CheckShardPlacements
- Break CheckShardPlacements into multiple functions (The most important
  is MarkFailedShardPlacements), so that we can get rid of the global
  CoordinatedTransactionUses2PC.
- Call MarkFailedShardPlacements in the router executor, so we mark
  shards as invalid and stop using them while inside transaction blocks.
2017-01-26 13:20:45 +02:00
Marco Slot f56454360c Mark failed placements as inactive immediately after COPY 2017-01-25 19:19:39 +03:00
Marco Slot b1626887d5 Don't mark placements inactive in COPY after successful connection 2017-01-25 19:19:38 +03:00
Marco Slot d0c76407b8 Set placement to inactive on connection failure in COPY 2017-01-25 19:19:38 +03:00
Marco Slot ba940a1de9 Use coordinator instead of schema node in terminology 2017-01-25 11:07:23 +01:00
Murat Tuncer d76f781ae4 Convert multi copy to use new connection api
This enables proper transactional behaviour for copy and relaxes some
restrictions like combining COPY with single-row modifications. It
also provides the basis for relaxing restrictions further, and for
optionally allowing connection caching.
2017-01-20 19:15:19 -08:00
Eren Basak 4def1ca696 Prevent COPY to reference tables from worker nodes 2017-01-18 17:38:01 +03:00
Andres Freund bdef35ac14 Query placementId in RemoteFinalizedShardPlacementList().
Not having the id in the ShardPlacement struct causes issues while
making copy use the placement aware connection management.
2017-01-17 13:27:26 -08:00
Onder Kalaci 1efa301ada Copy on reference tables should never mark placements invalid
This commit ensures that COPY does not mark any placement
of reference's state as INVALID in case of an error.
2017-01-12 02:43:41 +02:00
Onder Kalaci 9f0bd4cb36 Reference Table Support - Phase 1
With this commit, we implemented some basic features of reference tables.

To start with, a reference table is
  * a distributed table whithout a distribution column defined on it
  * the distributed table is single sharded
  * and the shard is replicated to all nodes

Reference tables follows the same code-path with a single sharded
tables. Thus, broadcast JOINs are applicable to reference tables.
But, since the table is replicated to all nodes, table fetching is
not required any more.

Reference tables support the uniqueness constraints for any column.

Reference tables can be used in INSERT INTO .. SELECT queries with
the following rules:
  * If a reference table is in the SELECT part of the query, it is
    safe join with another reference table and/or hash partitioned
    tables.
  * If a reference table is in the INSERT part of the query, all
    other participating tables should be reference tables.

Reference tables follow the regular co-location structure. Since
all reference tables are single sharded and replicated to all nodes,
they are always co-located with each other.

Queries involving only reference tables always follows router planner
and executor.

Reference tables can have composite typed columns and there is no need
to create/define the necessary support functions.

All modification queries, master_* UDFs, EXPLAIN, DDLs, TRUNCATE,
sequences, transactions, COPY, schema support works on reference
tables as expected. Plus, all the pre-requisites associated with
distribution columns are dismissed.
2016-12-20 14:09:35 +02:00
Andres Freund a77cf36778 Use connection_management.c from within connection_cache.c.
This is a temporary step towards removing connection_cache.c.
2016-12-07 11:44:24 -08:00
Marco Slot a497e7178c Parallelise master_modify_multiple_shards 2016-10-19 08:33:08 +02:00
Eren Basak cee7b54e7c Add worker transaction and transaction recovery infrastructure 2016-10-18 14:18:14 +03:00
Marco Slot 33b7723530 Use UpdateShardPlacementState where appropriate 2016-10-07 11:59:20 -07:00
Andres Freund 982ad66753 Introduce placement IDs.
So far placements were assigned an Oid, but that was just used to track
insertion order. It also did so incompletely, as it was not preserved
across changes of the shard state. The behaviour around oid wraparound
was also not entirely as intended.

The newly introduced, explicitly assigned, IDs are preserved across
shard-state changes.

The prime goal of this change is not to improve ordering of task
assignment policies, but to make it easier to reference shards.  The
newly introduced UpdateShardPlacementState() makes use of that, and so
will the in-progress connection and transaction management changes.
2016-10-07 11:59:20 -07:00
Jason Petersen 74f4e0003b
Permit multiple DDL commands in a transaction
Three changes here to get to true multi-statement, multi-relation DDL
transactions (same functionality pre-5.2, with benefits of atomicity):

    1. Changed the multi-shard utility hook to always run (consistency
       with router executor hook, removes ad-hoc "installed" boolean)

    2. Change the global connection list in multi_shard_transaction to
       instead be a hash; update related functions to operate on global
       hash instead of local hash/global list

    3. Remove check within DDL code to prevent subsequent DDL commands;
       place unset/reset guard around call to ConnectToNode to permit
       connecting to additional nodes after DDL transaction has begun

In addition, code has been added to raise an error if a ROLLBACK TO
SAVEPOINT is attempted (similar to router executor), and comprehensive
tests execute all multi-DDL scenarios (full success, user ROLLBACK, any
actual errors (say, duplicate index), partial failure (duplicate index
on one node but not others), partial COMMIT (one node fails), and 2PC
partial PREPARE (one node fails)). Interleavings with other commands
(DML, \copy) are similarly all covered.
2016-09-08 22:35:55 -05:00
Jason Petersen 850c51947a
Re-permit DDL in transactions, selectively
Recent changes to DDL and transaction logic resulted in a "regression"
from the viewpoint of users. Previously, DDL commands were allowed in
multi-command transaction blocks, though they were not processed in any
actual transactional manner. We improved the atomicity of our DDL code,
but added a restriction that DDL commands themselves must not occur in
any BEGIN/END transaction block.

To give users back the original functionality (and improved atomicity)
we now keep track of whether a multi-command transaction has modified
data (DML) or schema (DDL). Interleaving the two modification types in
a single transaction is disallowed.

This first step simply permits a single DDL command in such a block,
admittedly an incomplete solution, but one which will permit us to add
full multi-DDL command support in a subsequent commit.
2016-08-30 20:37:19 -06:00