Commit Graph

57 Commits (ee0cd756485c32bf630a32a38c9e465d95aecabc)

Author SHA1 Message Date
Marco Slot fba93df4b0 Remove copy into new append shard logic 2021-11-07 21:01:40 +01:00
Önder Kalacı 3f726c72e0
When replication factor > 1, all modifications are done via 2PC (#5379)
With Citus 9.0, we introduced `citus.single_shard_commit_protocol` which
defaults to 2PC.

With this commit, we prevent any user to set it to 1PC and drop support
for `citus.single_shard_commit_protocol`.

Although this might add some overhead for users, it is already the default
behaviour (so less likely) and marking placements as INVALID is much
worse.
2021-10-20 01:39:03 -07:00
Marco Slot 096660d61d Remove master_apply_delete_command 2021-10-18 22:29:37 +02:00
Burak Velioglu 4355ba0a38
Add CREATE INDEX ... ON ONLY and ALTER INDEX ... ATTACH PARTITION (#4938 #4980)
- Add support for CRETE INDEX ... ON ONLY: Before that commit we were not sending "ONLY" option to the worker nodes at all. With this commit, "ONLY" parameter will be sent to the worker nodes if it is necessary. (#4938)

- Add support for ALTER INDEX ... ATTACH PARTITION: Attach child_index to parent_index by creating same inheritance on shard level in addition to table level. (#4980)
2021-08-13 13:12:45 +03:00
Onur Tirtir cacb76d2c6
Not mention citus local tables in error messages (#4579) 2021-01-27 12:36:53 +03:00
Naisila Puka 7b05777682
Add ALTER TABLE .. SET LOGGED/UNLOGGED support (#4486) 2021-01-11 20:39:06 +03:00
Ahmet Gedemenli f27649754b
Add alter index set statistics support (#4455)
* Add alter index set statistics support

* Use attNum instead of attName
2021-01-05 13:23:11 +03:00
Onur Tirtir a58a4395ab Extend citus local table utility command support
This commit brings following features:

Foreign key support from citus local tables to reference tables
* Foreign key support from reference tables to citus local tables
  (only with RESTRICT & NO ACTION behavior)
* ALTER TABLE ENABLE/DISABLE trigger command support
* CREATE/DROP/ALTER trigger command support

and disallows:
* ALTER TABLE ATTACH/DETACH PARTITION commands
* CREATE TABLE <postgres table> ATTACH PARTITION <citus local table>
  commands
* Foreign keys from postgres tables to citus local tables
  (the other way was already disallowed)

for citus local tables.
2020-09-09 11:50:55 +03:00
Önder Kalacı 40da78c6fd
Introduce the adaptive executor (#2798)
With this commit, we're introducing the Adaptive Executor. 


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

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



All the commits involved here:
* Initial unified executor prototype

* Latest changes

* Fix rebase conflicts to master branch

* Add missing variable for assertion

* Ensure that master_modify_multiple_shards() returns the affectedTupleCount

* Adjust intermediate result sizes

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

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

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

That might be useful under certain circumstances:

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

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

* For parameters to be resolved before using them

For the details, see PostgreSQL's copyParamList()

* Unified executor sorts the returning output

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

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

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

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

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

* Fix 2PC conditions for DDL tasks

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

* Support foreign keys to reference tables in unified executor

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

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

* Make sure to close the failed connections after the execution

* Improve comments

* Fix savepoints in unified executor.

* Rebuild the WaitEventSet only when necessary

* Unclaim connections on all errors.

* Improve failure handling for unified executor

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

Improve the logic to start a transaction block for distributed transactions

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

* Move waitEventSet to DistributedExecution

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

* Fix multi_insert_select test outputs

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

* Fix subeury_and_cte test

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

which is a lot clearer to the user.

* Fix intermediate_results test outputs

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

which makes a lot more sense.

* Fix multi_function_in_join test

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

* Fix multi_query_directory_cleanup test

The unified executor does not create any intermediate files.

* Fix with_transactions test

A test case that just started to work fine

* Fix multi_router_planner test outputs

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

which is a lot more clearer for the users

* Fix multi_router_planner_fast_path test

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

which is a lot more clearer for the users

* Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block

* Fix ordering in isolation_multi_shard_modify_vs_all

* Add executor locks to unified executor

* Make sure to allocate enought WaitEvents

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

* Fix rebase conflicts for master rebase

* Make sure that TRUNCATE relies on unified executor

* Implement true sequential execution for multi-row INSERTS

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

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

* Remove SESSION_LIFESPAN flag in unified_executor

* Apply failure test updates

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

* Unified executor honors citus.node_connection_timeout

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

And, as a side effect this fixes failure_connection_establishment
test.

* Properly increment/decrement pool size variables

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

* insert_select_executor goes through unified executor.

* Add missing file for task tracker

* Modify ExecuteTaskListExtended()'s signature

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

* Take partition locks correctly in unified executor

* Alternative implementation for force_max_query_parallelization

* Fix compile warnings in unified executor

* Fix style issues

* Decrement idleConnectionCount when idle connection is lost

* Always rebuild the wait event sets

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

* Make sure to allocate enough sized waitEventSet

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

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

* Implement SELECT FOR UPDATE on reference tables

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

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

* SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled

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

* Fix build error

* Fix the newConnectionCount calculation in ManageWorkerPool

* Fix rebase conflicts

* Fix minor test output differences

* Fix citus indent

* Remove duplicate sorts that is added with rebase

* Create distributed table via executor

* Fix wait flags in CheckConnectionReady

* failure_savepoints output for unified executor.

* failure_vacuum output (pg 10) for unified executor.

* Fix WaitEventSetWait timeout in unified executor

* Stabilize failure_truncate test output

* Add an ORDER BY to multi_upsert

* Fix regression test outputs after rebase to master

* Add executor.c comment

* Rename executor.c to adaptive_executor.c

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

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

* Implement a proper custom scan node for adaptive executor

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

* Add slow start interval to the executor

* Expose max_cached_connection_per_worker to user

* Do not start slow when there are cached connections

* Consider ExecutorSlowStartInterval in NextEventTimeout

* Fix memory issues with ReceiveResults().

* Disable executor via TaskExecutorType

* Make sure to execute the tests with the other executor

* Use task_executor_type to enable-disable adaptive executor

* Remove useless code

* Adjust the regression tests

* Add slow start regression test

* Rebase to master

* Fix test failures in adaptive executor.

* Rebase to master - 2

* Improve comments & debug messages

* Set force_max_query_parallelization in isolation_citus_dist_activity

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

* Adjust the default pool size

* Expand description of max_adaptive_executor_pool_size GUC

* Update warnings in FinishRemoteTransactionCommit()

* Improve session clean up at the end of execution

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

* Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all

* Add more ORDER BYs to multi_mx_partitioning
2019-06-28 14:04:40 +02:00
Hanefi Onaldi b3d897039a constraint validation regression tests 2018-11-26 14:04:51 +03:00
Nils Dijk 6cf4516fdb
fix \d change for indexes in pg11 2018-08-15 23:27:31 -06:00
Nils Dijk 2a9d47e1a6
fix pg11 tests 2018-08-15 23:27:31 -06:00
mehmet furkan şahin 6d0fbbace7 ALTER TABLE %s ADD COLUMN constraint check is added 2018-07-24 15:53:05 +03:00
mehmet furkan şahin 785a86ed0a Tests are updated to use create_distributed_table 2018-05-10 11:18:59 +03:00
Marco Slot 3d3c19a717
Improve messages for essential connection failures 2018-04-26 12:58:47 -06:00
velioglu 72dfe4a289 Adds colocation check to local join 2018-04-04 22:49:27 +03:00
Dimitri Fontaine c9760fbb64 Fix CREATE INDEX with storage options on distributed tables.
By sharing the implementation of the function AppendOptionListToString on
three call sites, we would expand an extra OPTIONS keyword in a create index
statement, and omit other bits of the specific syntax here.

This patch introduces an AppendStorageParametersToString() function that is
very similar to AppendOptionListToString() but handles WITH(a="foo",...)
syntax that is used in reloptions (aka Storage Parameters).

Fixes #1747.
2018-01-17 21:56:40 +01:00
Dimitri Fontaine 952da72c55 Implement ALTER TABLE|INDEX ... SET|RESET ().
PostgreSQL implements support for several relation kinds in a single
statement, such as in the AlterTableStmt case, which supports both tables
and indexes and more (see ATExecSetRelOptions in PostgreSQL source code file
src/backend/commands/tablecmds.c for an example of that).

As a consequence, this patch implements support for setting and resetting
storage parameters on both relation kinds.
2018-01-17 21:56:40 +01:00
Dimitri Fontaine 17266e3301 Implement ALTER INDEX ... RENAME TO ...
The command is now distributed among the shards when the table is
distributed. To that effect, we fill in the DDLJob's targetRelationId with
the OID of the table for which the index is defined, rather than the OID of
the index itself.
2018-01-17 21:56:40 +01:00
Dimitri Fontaine e010238280 Implement ALTER TABLE ... RENAME TO ...
The implementation was already mostly in place, but the code was protected
by a principled check against the operation. Turns out there's a nasty
concurrency bug though with long identifier names, much as in #1664.

To prevent deadlocks from happening, we could either review the DDL
transaction management in shards and placements, or we can simply reject
names with (NAMEDATALEN - 1) chars or more — that's because of the
PostgreSQL array types being created with a one-char prefix: '_'.
2018-01-11 13:21:24 +01:00
Marco Slot a9933deac6 Make real time executor work in transactions 2017-11-30 09:59:32 +03:00
Marco Slot f4ceea5a3d Enable 2PC by default 2017-11-22 11:26:58 +01:00
mehmet furkan şahin 314fc09d90 regression test shard_count is changed from 32 to 4 2017-11-20 12:47:49 +03:00
mehmet furkan şahin 61ae33dc7f ALTER TABLE .. REPLICA IDENTITY support is implemented 2017-10-26 13:44:28 +03:00
Hadi Moshayedi e5fbcf37dd Add Savepoint Support (#1539)
This change adds support for SAVEPOINT, ROLLBACK TO SAVEPOINT, and RELEASE SAVEPOINT.

When transaction connections are not established yet, savepoints are kept in a stack and sent to the worker when the connection is later established. After establishing connections, savepoint commands are sent as they arrive.

This change fixes #1493 .
2017-08-15 13:02:28 -04:00
Burak Yucesoy fddf9b3fcc Add distributed partitioned table support distributed table creation
With this PR, Citus starts to support all possible ways to create
distributed partitioned tables. These are;

- Distributing already created partitioning hierarchy
- CREATE TABLE ... PARTITION OF a distributed_table
- ALTER TABLE distributed_table ATTACH PARTITION non_distributed_table
- ALTER TABLE distributed_table ATTACH PARTITION distributed_table

We also support DETACHing partitions from partitioned tables and propogating
TRUNCATE and DDL commands to distributed partitioned tables.

This PR also refactors some parts of distributed table creation logic.
2017-08-09 10:01:35 +03:00
Hadi Moshayedi 8229a64fe8 Remove distributed tables' dependency on distribution key columns. (#1527)
This change removes distributed tables' dependency on distribution key columns. We already check that we cannot drop distribution key columns in ErrorIfUnsupportedAlterTableStmt() at multi_utility.c, so we don't need to have distributed table to distribution key column dependency to avoid dropping of distribution key column.

Furthermore, having this dependency causes some warnings in pg_dump --schema-only (See #866), which are not desirable.

This change also adds check to disallow drop of distribution keys when citus.enable_ddl_propagation is set to false. Regression tests are updated accordingly.
2017-08-03 10:07:04 -04: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
velioglu a1ea29ec2b Use placement connection to drop shards instead of node connection 2017-06-14 14:14:59 +03:00
velioglu 24d24db25c Implement ALTER TABLE ADD CONSTRAINT command 2017-04-20 15:02:33 +03:00
Jason Petersen 5272c2c44b
Enable distributed ALTER TABLE ... RENAME COLUMN
Pretty straightforward. Had some concerns about locking, but due to the
fact that all distributed operations use either some level of deparsing
or need to enumerate column names, they all block during any concurrent
column renames (due to the AccessExclusive lock).

In addition, I had some misgivings about permitting renames of the dis-
tribution column, but nothing bad comes from just allowing them.

Finally, I tried to trigger any sort of error using prepared statements
and could not trigger any errors not also exhibited by plain PostgreSQL
tables.
2017-04-18 22:47:48 -06:00
Marco Slot f838c83809 Remove redundant pg_dist_jobid_seq restarts in tests 2017-04-18 11:42:32 +02:00
Metin Doslu 54a277ff01 Add disable/enable trigger all support 2017-03-29 22:00:14 +03:00
Jason Petersen f181b24859
Move worker execution to after master, fix tests
Some tests relied on worker errors though local commands were invalid.
Fixed those by ensuring preconditions were met to have command work
correctly. Otherwise most test changes are related to slight changes
in local/remote error ordering.
2017-03-22 17:21:49 -06:00
Andres Freund 52358fe891 Initial temp table removal implementation 2017-03-14 12:09:49 +02:00
Murat Tuncer 72027f2eba Remove default clause from shard DDL when sequences are used 2017-03-01 17:32:48 +03: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 fa5e202403 Convert multi_shard_transaction.[ch] to new framework. 2016-12-12 15:18:12 -08:00
Burak Yucesoy 8d7cd4d746 Add Foreign Key Support to ALTER TABLE commands
With this PR, we add foreign key support to ALTER TABLE commands. For now,
we only support foreign constraint creation via ALTER TABLE query, if it
is only subcommand in ALTER TABLE subcommand list.

We also only allow foreign key creation if replication factor is 1.
2016-12-08 15:03:25 +02:00
Sumedh Pathak 0a0d4784b9 Change DDL error message to say "unsupported" instead of "supported" 2016-11-26 10:30:09 +01:00
Marco Slot 271b20a23e Parallelise DDL commands 2016-10-24 12:39:08 +02: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
Eren Başak 0322916700
Lowercase \copy to match PostgreSQL's style for local/psql-level functions 2016-08-22 11:31:26 -06:00
Eren Basak b513f1c911
Replace \stage With \copy on Regression Tests
Fixes #547

This change removes all references to \stage in the regression tests
and puts \COPY instead. Doing so changed shard counts, min/max
values on some test tables (lineitem, orders, etc.).
2016-08-22 11:31:26 -06:00
Eren Başak bb3893d0d8 Set 1PC as the Default Commit Protocol for DDL Commands
Fixes #679

This change sets the default commit protocol for distributed DDL
commands to '1pc'. If the user issues a distributed DDL command with
this default setting, then once in a session, a NOTICE message is
shown about using '2pc' being extra safe.
2016-07-29 16:42:55 +03:00
Burak Yucesoy bdff72ed75 Fix ALTER TABLE SET SCHEMA
Fixes #132

We hook into ALTER ... SET SCHEMA and warn out if user tries to change schema of a
distributed table.

We also hook into ALTER TABLE ALL IN TABLE SPACE statements and warn out if citus has
been loaded.
2016-07-22 17:52:40 +03:00
Eren 3eaff48114 Propagate DDL Commands with 2PC
Fixes #513

This change modifies the DDL Propagation logic so that DDL queries
are propagated via 2-Phase Commit protocol. This way, failures during
the execution of distributed DDL commands will not leave the table in
an intermediate state and the pending prepared transactions can be
commited manually.

DDL commands are not allowed inside other transaction blocks or functions.

DDL commands are performed with 2PC regardless of the value of
`citus.multi_shard_commit_protocol` parameter.

The workflow of the successful case is this:
1. Open individual connections to all shard placements and send `BEGIN`
2. Send `SELECT worker_apply_shard_ddl_command(<shardId>, <DDL Command>)`
to all connections, one by one, in a serial manner.
3. Send `PREPARE TRANSCATION <transaction_id>` to all connections.
4. Sedn `COMMIT` to all connections.

Failure cases:
- If a worker problem occurs before sending of all DDL commands is finished, then
all changes are rolled back.
- If a worker problem occurs after all DDL commands are sent but not after
`PREPARE TRANSACTION` commands are finished, then all changes are rolled back.
However, if a worker node is failed, then the prepared transactions in that worker
should be rolled back manually.
- If a worker problem occurs during `COMMIT PREPARED` statements are being sent,
then the prepared transactions on the failed workers should be commited manually.
- If master fails before the first 'PREPARE TRANSACTION' is sent, then nothing is
changed on workers.
- If master fails during `PREPARE TRANSACTION` commands are being sent, then the
prepared transactions on workers should be rolled back manually.
- If master fails during `COMMIT PREPARED` or `ROLLBACK PREPARED` commands are being
sent, then the remaining prepared transactions on the workers should be handled manually.

This change also helps with #480, since failed DDL changes no longer mark
failed placements as inactive.
2016-07-19 10:44:11 +03:00
Jason Petersen 48f4e5d1a5
Make ReportRemoteError's CONTEXT style-compliant
There's not a ton of documentation about what CONTEXT lines should look
like, but this seems like the most dominant pattern. Similarly, users
should expect lowercase, non-period strings.
2016-06-07 12:47:16 -06:00
Metin Doslu 15eed396b3 Update ereport format 2016-06-07 15:58:32 +03:00