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
pull/2812/head
Önder Kalacı 2019-06-28 14:04:40 +02:00 committed by GitHub
parent 2d57899130
commit 40da78c6fd
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
166 changed files with 22054 additions and 899 deletions

View File

@ -72,6 +72,35 @@ jobs:
- {attach_workspace: {at: .}}
- {run: {name: 'Install and Test (check-multi)', command: 'install-and-test-ext check-multi'}}
- {codecov/upload: {flags: 'test_11,multi'}}
test-11_check-non-adaptive-multi:
docker:
- {image: 'citusdata/exttester-11:latest'}
working_directory: /home/circleci/project
steps:
- {attach_workspace: {at: .}}
- {run: {name: 'Install and Test (check-non-adaptive-executors)', command: 'install-and-test-ext check-multi-non-adaptive'}}
- {codecov/upload: {flags: 'test_11,multi'}}
test-11_check-non-adaptive-failure:
docker:
- {image: 'citusdata/failtester-11:latest'}
working_directory: /home/circleci/project
steps:
- {attach_workspace: {at: .}}
- {run: {name: 'Install and Test (check-non-adaptive-executors)', command: 'install-and-test-ext check-failure-non-adaptive'}}
- {codecov/upload: {flags: 'test_11,failure'}}
test-11_check-non-adaptive-isolation:
docker:
- {image: 'citusdata/exttester-11:latest'}
working_directory: /home/circleci/project
steps:
- {attach_workspace: {at: .}}
- {run: {name: 'Install and Test (check-non-adaptive-executors)', command: 'install-and-test-ext check-isolation-non-adaptive'}}
- {codecov/upload: {flags: 'test_11,isolation'}}
test-11_check-tt-van-mx:
docker:
- {image: 'citusdata/exttester-11:latest'}
@ -130,3 +159,7 @@ workflows:
- {test-11_check-iso-work-fol: {requires: [build]}}
- {test-11_check-fol: {requires: [build]}}
- {test-11_check-failure: {requires: [build]}}
- {test-11_check-non-adaptive-multi: {requires: [build]}}
- {test-11_check-non-adaptive-failure: {requires: [build]}}
- {test-11_check-non-adaptive-isolation: {requires: [build]}}

View File

@ -124,13 +124,13 @@ typedef struct CopyPlacementState CopyPlacementState;
* later, when copy ends or a switch-over happens. See CitusSendTupleToPlacements()
* for more details.
*
* This is done so we are compatible with executor.c. If a previous command
* in the current transaction has been executed using executor.c, then
* This is done so we are compatible with adaptive_executor. If a previous command
* in the current transaction has been executed using adaptive_executor.c, then
* CopyGetPlacementConnection() might return the same connection for multiple
* placements. We support that case by the buffering mechanism described above.
*
* If no previous command in the current transaction has used executor.c, then
* CopyGetPlacementConnection() returns one connection per placement and no
* If no previous command in the current transaction has used adaptive_executor.c,
* then CopyGetPlacementConnection() returns one connection per placement and no
* buffering happens and we put the copy data directly on connection.
*/
typedef struct CopyConnectionState

View File

@ -556,6 +556,7 @@ static void
ExecuteDistributedDDLJob(DDLJob *ddlJob)
{
bool shouldSyncMetadata = ShouldSyncTableMetadata(ddlJob->targetRelationId);
int targetPoolSize = ddlJob->executeSequentially ? 1 : MaxAdaptiveExecutorPoolSize;
EnsureCoordinator();
EnsurePartitionTableNotReplicated(ddlJob->targetRelationId);
@ -580,15 +581,9 @@ ExecuteDistributedDDLJob(DDLJob *ddlJob)
SendCommandToWorkers(WORKERS_WITH_METADATA, (char *) ddlJob->commandString);
}
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION ||
ddlJob->executeSequentially)
{
ExecuteModifyTasksSequentiallyWithoutResults(ddlJob->taskList, CMD_UTILITY);
}
else
{
ExecuteModifyTasksWithoutResults(ddlJob->taskList);
}
/* use adaptive executor when enabled */
ExecuteUtilityTaskListWithoutResults(ddlJob->taskList, targetPoolSize,
ddlJob->executeSequentially);
}
else
{
@ -599,7 +594,9 @@ ExecuteDistributedDDLJob(DDLJob *ddlJob)
PG_TRY();
{
ExecuteModifyTasksWithoutResults(ddlJob->taskList);
/* use adaptive executor when enabled */
ExecuteUtilityTaskListWithoutResults(ddlJob->taskList, targetPoolSize,
ddlJob->executeSequentially);
if (shouldSyncMetadata)
{

View File

@ -74,6 +74,7 @@ ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand)
{
List *vacuumColumnList = NIL;
List *taskList = NIL;
int targetPoolSize = MaxAdaptiveExecutorPoolSize;
/*
* VACUUM commands cannot run inside a transaction block, so we use
@ -92,8 +93,8 @@ ProcessVacuumStmt(VacuumStmt *vacuumStmt, const char *vacuumCommand)
vacuumColumnList = VacuumColumnList(vacuumStmt, relationIndex);
taskList = VacuumTaskList(relationId, vacuumStmt->options, vacuumColumnList);
ExecuteModifyTasksWithoutResults(taskList);
/* use adaptive executor when enabled */
ExecuteUtilityTaskListWithoutResults(taskList, targetPoolSize, false);
executedVacuumCount++;
}
relationIndex++;

View File

@ -67,7 +67,6 @@ typedef struct MultiConnectionPollState
static bool MultiConnectionStatePoll(MultiConnectionPollState *connectionState);
static WaitEventSet * WaitEventSetFromMultiConnectionStates(List *connections,
int *waitCount);
static long DeadlineTimestampTzToTimeout(TimestampTz deadline);
static void CloseNotReadyMultiConnectionStates(List *connectionStates);
static uint32 MultiConnectionStateEventMask(MultiConnectionPollState *connectionState);
@ -334,6 +333,8 @@ StartNodeUserDatabaseConnection(uint32 flags, const char *hostname, int32 port,
connection = StartConnectionEstablishment(&key);
dlist_push_tail(entry->connections, &connection->connectionNode);
entry->connectionCount++;
ResetShardPlacementAssociation(connection);
return connection;
@ -745,6 +746,17 @@ FinishConnectionListEstablishment(List *multiConnectionList)
uint32 eventMask = MultiConnectionStateEventMask(connectionState);
ModifyWaitEvent(waitEventSet, event->pos, eventMask, NULL);
}
/*
* The state has changed to connected, update the connection's
* state as well.
*/
if (connectionState->phase == MULTI_CONNECTION_PHASE_CONNECTED)
{
MultiConnection *connection = connectionState->connection;
connection->connectionState = MULTI_CONNECTION_CONNECTED;
}
}
}
@ -784,7 +796,7 @@ FinishConnectionListEstablishment(List *multiConnectionList)
* before the deadline provided as an argument will be reached. The outcome can be used to
* pass to the Wait of an EventSet to make sure it returns after the timeout has passed.
*/
static long
long
DeadlineTimestampTzToTimeout(TimestampTz deadline)
{
long secs = 0;
@ -1053,6 +1065,8 @@ AfterXactHostConnectionHandling(ConnectionHashEntry *entry, bool isCommit)
cachedConnectionCount++;
}
}
entry->connectionCount = cachedConnectionCount;
}
@ -1158,3 +1172,29 @@ TrimLogLevel(const char *message)
return chompedMessage + n;
}
/*
* NodeConnectionCount gets the number of connections to the given node
* for the current username and database.
*/
int
NodeConnectionCount(char *hostname, int port)
{
ConnectionHashKey key;
ConnectionHashEntry *entry = NULL;
bool found = false;
strlcpy(key.hostname, hostname, MAX_NODE_LENGTH);
key.port = port;
strlcpy(key.user, CurrentUserName(), NAMEDATALEN);
strlcpy(key.database, CurrentDatabaseName(), NAMEDATALEN);
entry = (ConnectionHashEntry *) hash_search(ConnectionHash, &key, HASH_FIND, &found);
if (!found)
{
return 0;
}
return entry->connectionCount;
}

View File

@ -183,8 +183,7 @@ static HTAB *ConnectionShardHash;
static MultiConnection * FindPlacementListConnection(int flags, List *placementAccessList,
const char *userName,
List **placementEntryList);
const char *userName);
static ConnectionPlacementHashEntry * FindOrCreatePlacementEntry(
ShardPlacement *placement);
static bool CanUseExistingConnection(uint32 flags, const char *userName,
@ -285,9 +284,6 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
const char *userName)
{
char *freeUserName = NULL;
ListCell *placementAccessCell = NULL;
List *placementEntryList = NIL;
ListCell *placementEntryCell = NULL;
MultiConnection *chosenConnection = NULL;
if (userName == NULL)
@ -295,8 +291,7 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
userName = freeUserName = CurrentUserName();
}
chosenConnection = FindPlacementListConnection(flags, placementAccessList, userName,
&placementEntryList);
chosenConnection = FindPlacementListConnection(flags, placementAccessList, userName);
if (chosenConnection == NULL)
{
/* use the first placement from the list to extract nodename and nodeport */
@ -337,28 +332,63 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
}
}
/*
* Now that a connection has been chosen, initialise or update the connection
* references for all placements.
*/
forboth(placementAccessCell, placementAccessList,
placementEntryCell, placementEntryList)
/* remember which connection we're going to use to access the placements */
AssignPlacementListToConnection(placementAccessList, chosenConnection);
if (freeUserName)
{
pfree(freeUserName);
}
return chosenConnection;
}
/*
* AssignPlacementListToConnection assigns a set of shard placement accesses to a
* given connection, meaning that connection must be used for all (conflicting)
* accesses of the same shard placements to make sure reads see writes and to
* make sure we don't take conflicting locks.
*/
void
AssignPlacementListToConnection(List *placementAccessList, MultiConnection *connection)
{
ListCell *placementAccessCell = NULL;
char *userName = connection->user;
foreach(placementAccessCell, placementAccessList)
{
ShardPlacementAccess *placementAccess =
(ShardPlacementAccess *) lfirst(placementAccessCell);
ShardPlacement *placement = placementAccess->placement;
ShardPlacementAccessType accessType = placementAccess->accessType;
ConnectionPlacementHashEntry *placementEntry =
(ConnectionPlacementHashEntry *) lfirst(placementEntryCell);
ConnectionReference *placementConnection = placementEntry->primaryConnection;
if (placementConnection->connection == chosenConnection)
ConnectionPlacementHashEntry *placementEntry = NULL;
ConnectionReference *placementConnection = NULL;
if (placement->shardId == INVALID_SHARD_ID)
{
/*
* When a SELECT prunes down to 0 shard, we use a dummy placement
* which is only used to route the query to a worker node, but
* the SELECT doesn't actually access any shard placement.
*
* FIXME: this can be removed if we evaluate empty SELECTs locally.
*/
continue;
}
placementEntry = FindOrCreatePlacementEntry(placement);
placementConnection = placementEntry->primaryConnection;
if (placementConnection->connection == connection)
{
/* using the connection that was already assigned to the placement */
}
else if (placementConnection->connection == NULL)
{
/* placement does not have a connection assigned yet */
placementConnection->connection = chosenConnection;
placementConnection->connection = connection;
placementConnection->hadDDL = false;
placementConnection->hadDML = false;
placementConnection->userName = MemoryContextStrdup(TopTransactionContext,
@ -366,7 +396,7 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
placementConnection->placementId = placementAccess->placement->placementId;
/* record association with connection */
dlist_push_tail(&chosenConnection->referencedPlacements,
dlist_push_tail(&connection->referencedPlacements,
&placementConnection->connectionNode);
}
else
@ -384,7 +414,7 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
* accessing the placement. However, we do register that it exists in
* hasSecondaryConnections.
*/
placementConnection->connection = chosenConnection;
placementConnection->connection = connection;
placementConnection->userName = MemoryContextStrdup(TopTransactionContext,
userName);
@ -392,7 +422,7 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
Assert(!placementConnection->hadDML);
/* record association with connection */
dlist_push_tail(&chosenConnection->referencedPlacements,
dlist_push_tail(&connection->referencedPlacements,
&placementConnection->connectionNode);
}
@ -421,14 +451,10 @@ StartPlacementListConnection(uint32 flags, List *placementAccessList,
{
placementConnection->hadDML = true;
}
}
if (freeUserName)
{
pfree(freeUserName);
/* record the relation access mapping */
AssociatePlacementAccessWithRelation(placement, accessType);
}
return chosenConnection;
}
@ -443,7 +469,6 @@ GetConnectionIfPlacementAccessedInXact(int flags, List *placementAccessList,
{
MultiConnection *connection = NULL;
char *freeUserName = NULL;
List *placementEntryList = NIL;
if (userName == NULL)
{
@ -451,7 +476,7 @@ GetConnectionIfPlacementAccessedInXact(int flags, List *placementAccessList,
}
connection = FindPlacementListConnection(flags, placementAccessList,
userName, &placementEntryList);
userName);
if (freeUserName != NULL)
{
@ -475,12 +500,10 @@ GetConnectionIfPlacementAccessedInXact(int flags, List *placementAccessList,
* function throws an error.
*
* The function returns the connection that needs to be used, if such a connection
* exists, and the current placement entries for all placements in the placement
* access list.
* exists.
*/
static MultiConnection *
FindPlacementListConnection(int flags, List *placementAccessList, const char *userName,
List **placementEntryList)
FindPlacementListConnection(int flags, List *placementAccessList, const char *userName)
{
bool foundModifyingConnection = false;
ListCell *placementAccessCell = NULL;
@ -678,8 +701,6 @@ FindPlacementListConnection(int flags, List *placementAccessList, const char *us
Assert(accessType != PLACEMENT_ACCESS_DDL);
}
*placementEntryList = lappend(*placementEntryList, placementEntry);
/* record the relation access mapping */
AssociatePlacementAccessWithRelation(placement, accessType);
}
@ -846,6 +867,42 @@ ConnectionAccessedDifferentPlacement(MultiConnection *connection,
}
/*
* ConnectionModifiedPlacement returns true if any DML or DDL is executed over
* the connection on any placement/table.
*/
bool
ConnectionModifiedPlacement(MultiConnection *connection)
{
dlist_iter placementIter;
if (connection->remoteTransaction.transactionState == REMOTE_TRANS_INVALID)
{
/*
* When StartPlacementListConnection() is called, we set the
* hadDDL/hadDML even before the actual command is sent to
* remote nodes. And, if this function is called at that
* point, we should not assume that the connection has already
* done any modifications.
*/
return false;
}
dlist_foreach(placementIter, &connection->referencedPlacements)
{
ConnectionReference *connectionReference =
dlist_container(ConnectionReference, connectionNode, placementIter.cur);
if (connectionReference->hadDDL || connectionReference->hadDML)
{
return true;
}
}
return false;
}
/*
* ConnectionUsedForAnyPlacements returns true if the connection
* has not been associated with any placement.

File diff suppressed because it is too large Load Diff

View File

@ -29,6 +29,7 @@
/* functions for creating custom scan nodes */
static Node * AdaptiveExecutorCreateScan(CustomScan *scan);
static Node * RealTimeCreateScan(CustomScan *scan);
static Node * TaskTrackerCreateScan(CustomScan *scan);
static Node * RouterCreateScan(CustomScan *scan);
@ -36,12 +37,17 @@ static Node * CoordinatorInsertSelectCreateScan(CustomScan *scan);
static Node * DelayedErrorCreateScan(CustomScan *scan);
/* functions that are common to different scans */
static void CitusSelectBeginScan(CustomScanState *node, EState *estate, int eflags);
static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags);
static void CitusEndScan(CustomScanState *node);
static void CitusReScan(CustomScanState *node);
/* create custom scan methods for all executors */
CustomScanMethods AdaptiveExecutorCustomScanMethods = {
"Citus Adaptive",
AdaptiveExecutorCreateScan
};
CustomScanMethods RealTimeCustomScanMethods = {
"Citus Real-Time",
RealTimeCreateScan
@ -71,9 +77,18 @@ CustomScanMethods DelayedErrorCustomScanMethods = {
/*
* Define executor methods for the different executor types.
*/
static CustomExecMethods AdaptiveExecutorCustomExecMethods = {
.CustomName = "AdaptiveExecutorScan",
.BeginCustomScan = CitusBeginScan,
.ExecCustomScan = CitusExecScan,
.EndCustomScan = CitusEndScan,
.ReScanCustomScan = CitusReScan,
.ExplainCustomScan = CitusExplainScan
};
static CustomExecMethods RealTimeCustomExecMethods = {
.CustomName = "RealTimeScan",
.BeginCustomScan = CitusSelectBeginScan,
.BeginCustomScan = CitusBeginScan,
.ExecCustomScan = RealTimeExecScan,
.EndCustomScan = CitusEndScan,
.ReScanCustomScan = CitusReScan,
@ -82,7 +97,7 @@ static CustomExecMethods RealTimeCustomExecMethods = {
static CustomExecMethods TaskTrackerCustomExecMethods = {
.CustomName = "TaskTrackerScan",
.BeginCustomScan = CitusSelectBeginScan,
.BeginCustomScan = CitusBeginScan,
.ExecCustomScan = TaskTrackerExecScan,
.EndCustomScan = CitusEndScan,
.ReScanCustomScan = CitusReScan,
@ -91,7 +106,7 @@ static CustomExecMethods TaskTrackerCustomExecMethods = {
static CustomExecMethods RouterModifyCustomExecMethods = {
.CustomName = "RouterModifyScan",
.BeginCustomScan = CitusModifyBeginScan,
.BeginCustomScan = CitusBeginScan,
.ExecCustomScan = RouterModifyExecScan,
.EndCustomScan = CitusEndScan,
.ReScanCustomScan = CitusReScan,
@ -100,7 +115,7 @@ static CustomExecMethods RouterModifyCustomExecMethods = {
static CustomExecMethods RouterSelectCustomExecMethods = {
.CustomName = "RouterSelectScan",
.BeginCustomScan = CitusSelectBeginScan,
.BeginCustomScan = CitusBeginScan,
.ExecCustomScan = RouterSelectExecScan,
.EndCustomScan = CitusEndScan,
.ReScanCustomScan = CitusReScan,
@ -109,7 +124,7 @@ static CustomExecMethods RouterSelectCustomExecMethods = {
static CustomExecMethods CoordinatorInsertSelectCustomExecMethods = {
.CustomName = "CoordinatorInsertSelectScan",
.BeginCustomScan = CitusSelectBeginScan,
.BeginCustomScan = CitusBeginScan,
.ExecCustomScan = CoordinatorInsertSelectExecScan,
.EndCustomScan = CitusEndScan,
.ReScanCustomScan = CitusReScan,
@ -123,6 +138,7 @@ static CustomExecMethods CoordinatorInsertSelectCustomExecMethods = {
void
RegisterCitusCustomScanMethods(void)
{
RegisterCustomScanMethods(&AdaptiveExecutorCustomScanMethods);
RegisterCustomScanMethods(&RealTimeCustomScanMethods);
RegisterCustomScanMethods(&TaskTrackerCustomScanMethods);
RegisterCustomScanMethods(&RouterCustomScanMethods);
@ -131,6 +147,76 @@ RegisterCitusCustomScanMethods(void)
}
/*
* CitusBeginScan sets the coordinator backend initiated by Citus for queries using
* that function as the BeginCustomScan callback.
*
* The function also handles modification scan actions.
*/
static void
CitusBeginScan(CustomScanState *node, EState *estate, int eflags)
{
CitusScanState *scanState = NULL;
DistributedPlan *distributedPlan = NULL;
MarkCitusInitiatedCoordinatorBackend();
scanState = (CitusScanState *) node;
distributedPlan = scanState->distributedPlan;
if (distributedPlan->operation == CMD_SELECT ||
distributedPlan->insertSelectSubquery != NULL)
{
/* no more action required */
return;
}
CitusModifyBeginScan(node, estate, eflags);
}
/*
* CitusExecScan is called when a tuple is pulled from a custom scan.
* On the first call, it executes the distributed query and writes the
* results to a tuple store. The postgres executor calls this function
* repeatedly to read tuples from the tuple store.
*/
TupleTableSlot *
CitusExecScan(CustomScanState *node)
{
CitusScanState *scanState = (CitusScanState *) node;
TupleTableSlot *resultSlot = NULL;
if (!scanState->finishedRemoteScan)
{
AdaptiveExecutor(node);
scanState->finishedRemoteScan = true;
}
resultSlot = ReturnTupleFromTuplestore(scanState);
return resultSlot;
}
/*
* AdaptiveExecutorCreateScan creates the scan state for the adaptive executor.
*/
static Node *
AdaptiveExecutorCreateScan(CustomScan *scan)
{
CitusScanState *scanState = palloc0(sizeof(CitusScanState));
scanState->executorType = MULTI_EXECUTOR_ADAPTIVE;
scanState->customScanState.ss.ps.type = T_CustomScanState;
scanState->distributedPlan = GetDistributedPlan(scan);
scanState->customScanState.methods = &AdaptiveExecutorCustomExecMethods;
return (Node *) scanState;
}
/*
* RealTimeCreateScan creates the scan state for real-time executor queries.
*/
@ -250,17 +336,6 @@ DelayedErrorCreateScan(CustomScan *scan)
}
/*
* CitusSelectBeginScan sets the coordinator backend initiated by Citus for queries using
* that function as the BeginCustomScan callback.
*/
static void
CitusSelectBeginScan(CustomScanState *node, EState *estate, int eflags)
{
MarkCitusInitiatedCoordinatorBackend();
}
/*
* CitusEndScan is used to clean up tuple store of the given custom scan state.
*/

View File

@ -9,6 +9,7 @@
*/
#include "postgres.h"
#include "miscadmin.h"
#include "distributed/commands/multi_copy.h"
#include "distributed/insert_select_executor.h"
@ -98,7 +99,6 @@ CoordinatorInsertSelectExecScan(CustomScanState *node)
List *taskList = workerJob->taskList;
List *prunedTaskList = NIL;
bool hasReturning = distributedPlan->hasReturning;
bool isModificationQuery = true;
shardStateHash = ExecuteSelectIntoColocatedIntermediateResults(
targetRelationId,
@ -128,15 +128,32 @@ CoordinatorInsertSelectExecScan(CustomScanState *node)
if (prunedTaskList != NIL)
{
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION)
if (TaskExecutorType != MULTI_EXECUTOR_ADAPTIVE)
{
ExecuteModifyTasksSequentially(scanState, prunedTaskList,
CMD_INSERT, hasReturning);
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION)
{
ExecuteModifyTasksSequentially(scanState, prunedTaskList,
CMD_INSERT, hasReturning);
}
else
{
ExecuteMultipleTasks(scanState, prunedTaskList, true,
hasReturning);
}
}
else
{
ExecuteMultipleTasks(scanState, prunedTaskList, isModificationQuery,
hasReturning);
TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState);
bool randomAccess = true;
bool interTransactions = false;
Assert(scanState->tuplestorestate == NULL);
scanState->tuplestorestate =
tuplestore_begin_heap(randomAccess, interTransactions, work_mem);
ExecuteTaskListExtended(CMD_INSERT, prunedTaskList,
tupleDescriptor, scanState->tuplestorestate,
hasReturning, MaxAdaptiveExecutorPoolSize);
}
if (SortReturning && hasReturning)

View File

@ -100,14 +100,9 @@ static List * BuildPlacementAccessList(int32 groupId, List *relationShardList,
static List * GetModifyConnections(Task *task, bool markCritical);
static int64 ExecuteModifyTasks(List *taskList, bool expectResults,
ParamListInfo paramListInfo, CitusScanState *scanState);
static void AcquireExecutorShardLock(Task *task, CmdType commandType);
static void AcquireExecutorMultiShardLocks(List *taskList);
static bool RequiresConsistentSnapshot(Task *task);
static void RouterMultiModifyExecScan(CustomScanState *node);
static void RouterSequentialModifyExecScan(CustomScanState *node);
static void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo,
Oid **parameterTypes,
const char ***parameterValues);
static bool SendQueryInSingleRowMode(MultiConnection *connection, char *query,
ParamListInfo paramListInfo);
static bool StoreQueryResult(CitusScanState *scanState, MultiConnection *connection, bool
@ -157,7 +152,7 @@ AcquireMetadataLocks(List *taskList)
* to communicate that the application is only generating commutative
* UPDATE/DELETE/UPSERT commands and exclusive locks are unnecessary.
*/
static void
void
AcquireExecutorShardLock(Task *task, CmdType commandType)
{
LOCKMODE lockMode = NoLock;
@ -349,7 +344,7 @@ AcquireExecutorShardLock(Task *task, CmdType commandType)
* RowExclusiveLock, which is normally obtained by single-shard, commutative
* writes.
*/
static void
void
AcquireExecutorMultiShardLocks(List *taskList)
{
ListCell *taskCell = NULL;
@ -485,8 +480,6 @@ RequiresConsistentSnapshot(Task *task)
*
* The function also checks the validity of the given custom scan node and
* gets locks on the shards involved in the task list of the distributed plan.
*
* It also sets the backend as initiated by Citus.
*/
void
CitusModifyBeginScan(CustomScanState *node, EState *estate, int eflags)
@ -497,8 +490,6 @@ CitusModifyBeginScan(CustomScanState *node, EState *estate, int eflags)
Query *jobQuery = NULL;
List *taskList = NIL;
MarkCitusInitiatedCoordinatorBackend();
/*
* We must not change the distributed plan since it may be reused across multiple
* executions of a prepared statement. Instead we create a deep copy that we only
@ -773,17 +764,48 @@ TaskListRequires2PC(List *taskList)
* Can't we always rely on anchorShardId?
*/
anchorShardId = task->anchorShardId;
if (ReferenceTableShardId(anchorShardId))
if (anchorShardId != INVALID_SHARD_ID && ReferenceTableShardId(anchorShardId))
{
return true;
}
multipleTasks = list_length(taskList) > 1;
if (multipleTasks && MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC)
if (!ReadOnlyTask(task->taskType) &&
multipleTasks && MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC)
{
return true;
}
if (task->taskType == DDL_TASK)
{
if (MultiShardCommitProtocol == COMMIT_PROTOCOL_2PC ||
task->replicationModel == REPLICATION_MODEL_2PC)
{
return true;
}
}
return false;
}
/*
* ReadOnlyTask returns true if the input task does a read-only operation
* on the database.
*/
bool
ReadOnlyTask(TaskType taskType)
{
if (taskType == ROUTER_TASK || taskType == SQL_TASK)
{
/*
* TODO: We currently do not execute modifying CTEs via ROUTER_TASK/SQL_TASK.
* When we implement it, we should either not use the mentioned task types for
* modifying CTEs detect them here.
*/
return true;
}
return false;
}
@ -1713,7 +1735,7 @@ SendQueryInSingleRowMode(MultiConnection *connection, char *query,
* ExtractParametersFromParamListInfo extracts parameter types and values from
* the given ParamListInfo structure, and fills parameter type and value arrays.
*/
static void
void
ExtractParametersFromParamListInfo(ParamListInfo paramListInfo, Oid **parameterTypes,
const char ***parameterValues)
{

View File

@ -20,6 +20,7 @@
#include <unistd.h>
#include "distributed/multi_client_executor.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_physical_planner.h"
#include "distributed/multi_resowner.h"
#include "distributed/multi_server_executor.h"
@ -28,7 +29,7 @@
#include "utils/lsyscache.h"
int RemoteTaskCheckInterval = 100; /* per cycle sleep interval in millisecs */
int TaskExecutorType = MULTI_EXECUTOR_REAL_TIME; /* distributed executor type */
int TaskExecutorType = MULTI_EXECUTOR_ADAPTIVE; /* distributed executor type */
bool BinaryMasterCopyFormat = false; /* copy data from workers in binary format */
bool EnableRepartitionJoins = false;
@ -74,11 +75,23 @@ JobExecutorType(DistributedPlan *distributedPlan)
ereport(DEBUG2, (errmsg("Plan is router executable")));
}
}
if (TaskExecutorType == MULTI_EXECUTOR_ADAPTIVE)
{
return TaskExecutorType;
}
return MULTI_EXECUTOR_ROUTER;
}
if (distributedPlan->insertSelectSubquery != NULL)
{
/*
* Even if adaptiveExecutorEnabled, we go through
* MULTI_EXECUTOR_COORDINATOR_INSERT_SELECT because
* the executor already knows how to handle adaptive
* executor when necessary.
*/
return MULTI_EXECUTOR_COORDINATOR_INSERT_SELECT;
}
@ -92,7 +105,6 @@ JobExecutorType(DistributedPlan *distributedPlan)
if (executorType == MULTI_EXECUTOR_REAL_TIME)
{
double reasonableConnectionCount = 0;
int dependedJobCount = 0;
/* if we need to open too many connections per worker, warn the user */
if (tasksPerNode >= MaxConnections)
@ -118,11 +130,15 @@ JobExecutorType(DistributedPlan *distributedPlan)
"setting citus.task_executor_type to "
"\"task-tracker\".")));
}
}
if (executorType == MULTI_EXECUTOR_REAL_TIME ||
executorType == MULTI_EXECUTOR_ADAPTIVE)
{
/* if we have repartition jobs with real time executor and repartition
* joins are not enabled, error out. Otherwise, switch to task-tracker
*/
dependedJobCount = list_length(job->dependedJobList);
int dependedJobCount = list_length(job->dependedJobList);
if (dependedJobCount > 0)
{
if (!EnableRepartitionJoins)

View File

@ -30,6 +30,7 @@
#include "distributed/master_protocol.h"
#include "distributed/metadata_cache.h"
#include "distributed/multi_join_order.h"
#include "distributed/multi_executor.h"
#include "distributed/multi_partitioning_utils.h"
#include "distributed/pg_dist_partition.h"
#include "distributed/pg_dist_shard.h"
@ -177,8 +178,14 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
workerNodeList = ActivePrimaryNodeList();
workerNodeList = SortList(workerNodeList, CompareWorkerNodes);
/* make sure we don't process cancel signals until all shards are created */
HOLD_INTERRUPTS();
/*
* Make sure we don't process cancel signals until all shards
* are created if the executor is not enabled.
*/
if (TaskExecutorType != MULTI_EXECUTOR_ADAPTIVE)
{
HOLD_INTERRUPTS();
}
workerNodeCount = list_length(workerNodeList);
if (replicationFactor > workerNodeCount)
@ -245,13 +252,17 @@ CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
CreateShardsOnWorkers(distributedTableId, insertedShardPlacements,
useExclusiveConnections, colocatedShard);
if (QueryCancelPending)
if (TaskExecutorType != MULTI_EXECUTOR_ADAPTIVE)
{
ereport(WARNING, (errmsg("cancel requests are ignored during shard creation")));
QueryCancelPending = false;
}
if (QueryCancelPending)
{
ereport(WARNING, (errmsg(
"cancel requests are ignored during shard creation")));
QueryCancelPending = false;
}
RESUME_INTERRUPTS();
RESUME_INTERRUPTS();
}
}

View File

@ -30,6 +30,7 @@
#include "distributed/commands.h"
#include "distributed/connection_management.h"
#include "distributed/distributed_planner.h"
#include "distributed/listutils.h"
#include "distributed/multi_client_executor.h"
#include "distributed/multi_router_executor.h"
#include "distributed/master_metadata_utility.h"
@ -58,6 +59,14 @@
/* Local functions forward declarations */
static void CreateShardsOnWorkersViaExecutor(Oid distributedRelationId,
List *shardPlacements,
bool useExclusiveConnection, bool
colocatedShard);
static void CreateShardsOnWorkersViaCommands(Oid distributedRelationId,
List *shardPlacements,
bool useExclusiveConnection, bool
colocatedShard);
static List * RelationShardListForShardCreate(ShardInterval *shardInterval);
static bool WorkerShardStats(ShardPlacement *placement, Oid relationId,
char *shardName, uint64 *shardSize,
@ -488,14 +497,115 @@ InsertShardPlacementRows(Oid relationId, int64 shardId, List *workerNodeList,
/*
* CreateShardsOnWorkers creates shards on worker nodes given the shard placements
* as a parameter. Function opens connections in transactional way. If the caller
* needs an exclusive connection (in case of distributing local table with data
* on it) or creating shards in a transaction, per placement connection is opened
* for each placement.
* as a parameter. Function branches into two: either use the executor or execute the
* commands one by one.
*/
void
CreateShardsOnWorkers(Oid distributedRelationId, List *shardPlacements,
bool useExclusiveConnection, bool colocatedShard)
{
if (TaskExecutorType == MULTI_EXECUTOR_ADAPTIVE)
{
CreateShardsOnWorkersViaExecutor(distributedRelationId, shardPlacements,
useExclusiveConnection, colocatedShard);
}
else
{
CreateShardsOnWorkersViaCommands(distributedRelationId, shardPlacements,
useExclusiveConnection, colocatedShard);
}
}
/*
* CreateShardsOnWorkersViaExecutor creates the shards via the executor. This means
* that it can adopt the number of connections required to create the shards.
*/
static void
CreateShardsOnWorkersViaExecutor(Oid distributedRelationId, List *shardPlacements, bool
useExclusiveConnection, bool colocatedShard)
{
bool includeSequenceDefaults = false;
List *ddlCommandList = GetTableDDLEvents(distributedRelationId,
includeSequenceDefaults);
List *foreignConstraintCommandList =
GetTableForeignConstraintCommands(distributedRelationId);
ListCell *shardPlacementCell = NULL;
int taskId = 1;
List *taskList = NIL;
int poolSize = 1;
foreach(shardPlacementCell, shardPlacements)
{
ShardPlacement *shardPlacement = (ShardPlacement *) lfirst(shardPlacementCell);
uint64 shardId = shardPlacement->shardId;
ShardInterval *shardInterval = LoadShardInterval(shardId);
int shardIndex = -1;
List *commandList = NIL;
Task *task = NULL;
List *relationShardList = RelationShardListForShardCreate(shardInterval);
if (colocatedShard)
{
shardIndex = ShardIndex(shardInterval);
}
commandList = WorkerCreateShardCommandList(distributedRelationId, shardIndex,
shardId, ddlCommandList,
foreignConstraintCommandList);
task = CitusMakeNode(Task);
task->jobId = INVALID_JOB_ID;
task->taskId = taskId++;
task->taskType = DDL_TASK;
task->queryString = StringJoin(commandList, ';');
task->replicationModel = REPLICATION_MODEL_INVALID;
task->dependedTaskList = NIL;
task->anchorShardId = shardId;
task->relationShardList = relationShardList;
task->taskPlacementList = list_make1(shardPlacement);
taskList = lappend(taskList, task);
}
if (useExclusiveConnection)
{
/*
* When the table has local data, we force max parallelization so data
* copy is done efficiently. We also prefer to use max parallelization
* when we're inside a transaction block because the user might execute
* compute heavy commands (e.g., load data or create index) later in the
* transaction block.
*/
SetLocalForceMaxQueryParallelization();
/*
* TODO: After we fix adaptive executor to record parallel access for
* ForceMaxQueryParallelization, we should remove this. This is just
* to force adaptive executor to record parallel access to relations.
*
* Adaptive executor uses poolSize to decide if it should record parallel
* access to relations or not, and it ignores ForceMaxQueryParallelization
* because of some complications in TRUNCATE.
*/
poolSize = MaxAdaptiveExecutorPoolSize;
}
ExecuteTaskList(CMD_UTILITY, taskList, poolSize);
}
/*
* CreateShardsOnWorkersViaCommands creates shards on worker nodes given the shard
* placements as a parameter. Function opens connections in transactional way. If the
* caller needs an exclusive connection (in case of distributing local table with data
* on it) or creating shards in a transaction, per placement connection is opened
* for each placement.
*/
static void
CreateShardsOnWorkersViaCommands(Oid distributedRelationId, List *shardPlacements,
bool useExclusiveConnection, bool colocatedShard)
{
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedRelationId);

View File

@ -77,14 +77,8 @@ citus_truncate_trigger(PG_FUNCTION_ARGS)
{
List *taskList = TruncateTaskList(relationId);
if (MultiShardConnectionType == SEQUENTIAL_CONNECTION)
{
ExecuteModifyTasksSequentiallyWithoutResults(taskList, CMD_UTILITY);
}
else
{
ExecuteModifyTasksWithoutResults(taskList);
}
ExecuteUtilityTaskListWithoutResults(taskList, MaxAdaptiveExecutorPoolSize,
false);
}
PG_RETURN_DATUM(PointerGetDatum(NULL));

View File

@ -979,6 +979,12 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan)
switch (executorType)
{
case MULTI_EXECUTOR_ADAPTIVE:
{
customScan->methods = &AdaptiveExecutorCustomScanMethods;
break;
}
case MULTI_EXECUTOR_REAL_TIME:
{
customScan->methods = &RealTimeCustomScanMethods;

View File

@ -296,6 +296,7 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
distributedPlan->masterQuery = NULL;
distributedPlan->routerExecutable = true;
distributedPlan->hasReturning = false;
distributedPlan->targetRelationId = targetRelationId;
if (list_length(originalQuery->returningList) > 0)
{

View File

@ -113,6 +113,7 @@ static void CreateSingleTaskRouterPlan(DistributedPlan *distributedPlan,
Query *query,
PlannerRestrictionContext *
plannerRestrictionContext);
static Oid ResultRelationOidForQuery(Query *query);
static bool IsTidColumn(Node *node);
static DeferredErrorMessage * MultiShardModifyQuerySupported(Query *originalQuery,
PlannerRestrictionContext *
@ -228,6 +229,7 @@ CreateModifyPlan(Query *originalQuery, Query *query,
distributedPlan->masterQuery = NULL;
distributedPlan->routerExecutable = true;
distributedPlan->hasReturning = false;
distributedPlan->targetRelationId = ResultRelationOidForQuery(query);
if (list_length(originalQuery->returningList) > 0)
{
@ -496,6 +498,19 @@ ModifyQueryResultRelationId(Query *query)
}
/*
* ResultRelationOidForQuery returns the OID of the relation this is modified
* by a given query.
*/
static Oid
ResultRelationOidForQuery(Query *query)
{
RangeTblEntry *resultRTE = rt_fetch(query->resultRelation, query->rtable);
return resultRTE->relid;
}
/*
* ExtractInsertRangeTableEntry returns the INSERT'ed table's range table entry.
* Note that the function expects and asserts that the input query be

View File

@ -109,6 +109,7 @@ static const struct config_enum_entry replication_model_options[] = {
};
static const struct config_enum_entry task_executor_type_options[] = {
{ "adaptive", MULTI_EXECUTOR_ADAPTIVE, false },
{ "real-time", MULTI_EXECUTOR_REAL_TIME, false },
{ "task-tracker", MULTI_EXECUTOR_TASK_TRACKER, false },
{ NULL, 0, false }
@ -613,6 +614,37 @@ RegisterCitusConfigVariables(void)
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.force_max_query_parallelization",
gettext_noop("Open as many connections as possible to maximize query "
"parallelization"),
gettext_noop("When enabled, Citus will force the executor to use "
"as many connections as possible while executing a "
"parallel distributed query. If not enabled, the executor"
"might choose to use less connections to optimize overall "
"query execution throughput. Internally, setting this true "
"will end up with using one connection per task."),
&ForceMaxQueryParallelization,
false,
PGC_USERSET,
GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomIntVariable(
"citus.executor_slow_start_interval",
gettext_noop("Time to wait between opening connections to the same worker node"),
gettext_noop("When the individual tasks of a multi-shard query take very "
"little time, they can often be finished over a single (often "
"already cached) connection. To avoid redundantly opening "
"additional connections, the executor waits between connection "
"attempts for the configured number of milliseconds. At the end "
"of the interval, it increases the number of connections it is "
"allowed to open next time."),
&ExecutorSlowStartInterval,
10, 0, INT_MAX,
PGC_USERSET,
GUC_UNIT_MS | GUC_NO_SHOW_ALL,
NULL, NULL, NULL);
DefineCustomBoolVariable(
"citus.enable_deadlock_prevention",
@ -724,6 +756,24 @@ RegisterCitusConfigVariables(void)
GUC_UNIT_KB,
NULL, NULL, NULL);
DefineCustomIntVariable(
"citus.max_adaptive_executor_pool_size",
gettext_noop("Sets the maximum number of connections per worker node used by "
"the adaptive executor to execute a multi-shard command"),
gettext_noop("The adaptive executor may open multiple connections per worker "
"node when running multi-shard commands to parallelize the command "
"across multiple cores on the worker. This setting specifies the "
"maximum number of connections it will open. The number of "
"connections is also bounded by the number of shards on the node. "
"This setting can be used to reduce the memory usage of a query "
"and allow a higher degree of concurrency when concurrent "
"multi-shard queries open too many connections to a worker."),
&MaxAdaptiveExecutorPoolSize,
16, 1, INT_MAX,
PGC_USERSET,
0,
NULL, NULL, NULL);
DefineCustomIntVariable(
"citus.max_worker_nodes_tracked",
gettext_noop("Sets the maximum number of worker nodes that are tracked."),
@ -776,7 +826,7 @@ RegisterCitusConfigVariables(void)
&MaxCachedConnectionsPerWorker,
1, 0, INT_MAX,
PGC_USERSET,
GUC_NO_SHOW_ALL,
0,
NULL, NULL, NULL);
DefineCustomIntVariable(
@ -928,7 +978,7 @@ RegisterCitusConfigVariables(void)
"queries that touch thousands of shards and/or that involve table "
"repartitioning."),
&TaskExecutorType,
MULTI_EXECUTOR_REAL_TIME,
MULTI_EXECUTOR_ADAPTIVE,
task_executor_type_options,
PGC_USERSET,
0,

View File

@ -126,6 +126,8 @@ StartRemoteTransactionBegin(struct MultiConnection *connection)
HandleRemoteTransactionConnectionError(connection, raiseErrors);
}
transaction->beginSent = true;
}
@ -301,17 +303,8 @@ FinishRemoteTransactionCommit(MultiConnection *connection)
if (transaction->transactionState == REMOTE_TRANS_1PC_COMMITTING)
{
if (transaction->transactionCritical)
{
ereport(WARNING, (errmsg("failed to commit critical transaction "
"on %s:%d, metadata is likely out of sync",
connection->hostname, connection->port)));
}
else
{
ereport(WARNING, (errmsg("failed to commit transaction on %s:%d",
connection->hostname, connection->port)));
}
ereport(WARNING, (errmsg("failed to commit transaction on %s:%d",
connection->hostname, connection->port)));
}
else if (transaction->transactionState == REMOTE_TRANS_2PC_COMMITTING)
{
@ -1255,6 +1248,9 @@ FinishRemoteTransactionSavepointRollback(MultiConnection *connection, SubTransac
PQclear(result);
ForgetResults(connection);
/* reset transaction state so the executor can accept next commands in transaction */
transaction->transactionState = REMOTE_TRANS_STARTED;
}

View File

@ -14,6 +14,7 @@
#include "port.h"
#include "utils/lsyscache.h"
#include "lib/stringinfo.h"
#include "distributed/listutils.h"
#include "nodes/pg_list.h"
#include "utils/memutils.h"
@ -154,3 +155,26 @@ ListToHashSet(List *itemList, Size keySize, bool isStringList)
return itemSet;
}
/*
* StringJoin gets a list of char * and then simply
* returns a newly allocated char * joined with the
* given delimiter.
*/
char *
StringJoin(List *stringList, char delimiter)
{
ListCell *stringCell = NULL;
StringInfo joinedString = makeStringInfo();
foreach(stringCell, stringList)
{
const char *command = lfirst(stringCell);
appendStringInfoString(joinedString, command);
appendStringInfoChar(joinedString, delimiter);
}
return joinedString->data;
}

View File

@ -27,6 +27,7 @@ typedef struct CitusScanState
/* custom scan methods for all executors */
extern CustomScanMethods AdaptiveExecutorCustomScanMethods;
extern CustomScanMethods RealTimeCustomScanMethods;
extern CustomScanMethods TaskTrackerCustomScanMethods;
extern CustomScanMethods RouterCustomScanMethods;

View File

@ -47,6 +47,14 @@ enum MultiConnectionMode
CONNECTION_PER_PLACEMENT = 1 << 3
};
typedef enum MultiConnectionState
{
MULTI_CONNECTION_INITIAL,
MULTI_CONNECTION_CONNECTING,
MULTI_CONNECTION_CONNECTED,
MULTI_CONNECTION_FAILED,
MULTI_CONNECTION_LOST
} MultiConnectionState;
/* declaring this directly above makes uncrustify go crazy */
typedef enum MultiConnectionMode MultiConnectionMode;
@ -62,6 +70,15 @@ typedef struct MultiConnection
/* underlying libpq connection */
struct pg_conn *pgConn;
/* state of the connection */
MultiConnectionState connectionState;
/* signal that the connection is ready for read/write */
bool ioReady;
/* whether to wait for read/write */
int waitFlags;
/* force the connection to be closed at the end of the transaction */
bool forceCloseAtTransactionEnd;
@ -111,6 +128,7 @@ typedef struct ConnectionHashEntry
{
ConnectionHashKey key;
dlist_head *connections;
int connectionCount;
} ConnectionHashEntry;
/* hash entry for cached connection parameters */
@ -173,12 +191,14 @@ extern MultiConnection * StartNodeUserDatabaseConnection(uint32 flags,
extern void CloseNodeConnectionsAfterTransaction(char *nodeName, int nodePort);
extern void CloseConnection(MultiConnection *connection);
extern void ShutdownConnection(MultiConnection *connection);
extern int NodeConnectionCount(char *nodeName, int nodePort);
/* dealing with a connection */
extern void FinishConnectionListEstablishment(List *multiConnectionList);
extern void FinishConnectionEstablishment(MultiConnection *connection);
extern void ClaimConnectionExclusively(MultiConnection *connection);
extern void UnclaimConnection(MultiConnection *connection);
extern long DeadlineTimestampTzToTimeout(TimestampTz deadline);
/* dealing with notice handler */
extern void SetCitusNoticeProcessor(MultiConnection *connection);

View File

@ -27,5 +27,6 @@ extern void ** PointerArrayFromList(List *pointerList);
extern ArrayType * DatumArrayToArrayType(Datum *datumArray, int datumCount,
Oid datumTypeId);
extern HTAB * ListToHashSet(List *pointerList, Size keySize, bool isStringList);
extern char * StringJoin(List *stringList, char delimiter);
#endif /* CITUS_LISTUTILS_H */

View File

@ -29,11 +29,23 @@ extern int MultiShardConnectionType;
extern bool WritableStandbyCoordinator;
extern bool ForceMaxQueryParallelization;
extern int MaxAdaptiveExecutorPoolSize;
extern int ExecutorSlowStartInterval;
extern void CitusExecutorStart(QueryDesc *queryDesc, int eflags);
extern void CitusExecutorRun(QueryDesc *queryDesc, ScanDirection direction, uint64 count,
bool execute_once);
extern TupleTableSlot * AdaptiveExecutor(CustomScanState *node);
extern uint64 ExecuteTaskListExtended(CmdType operation, List *taskList,
TupleDesc tupleDescriptor,
Tuplestorestate *tupleStore,
bool hasReturning, int targetPoolSize);
extern void ExecuteUtilityTaskListWithoutResults(List *taskList, int targetPoolSize,
bool forceSequentialExecution);
extern uint64 ExecuteTaskList(CmdType operation, List *taskList, int targetPoolSize);
extern TupleTableSlot * CitusExecScan(CustomScanState *node);
extern TupleTableSlot * ReturnTupleFromTuplestore(CitusScanState *scanState);
extern void LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob);
extern void ReadFileIntoTupleStore(char *fileName, char *copyFormat, TupleDesc
@ -47,6 +59,7 @@ extern void ExecuteQueryIntoDestReceiver(Query *query, ParamListInfo params,
extern void ExecutePlanIntoDestReceiver(PlannedStmt *queryPlan, ParamListInfo params,
DestReceiver *dest);
extern void SetLocalMultiShardModifyModeToSequential(void);
extern void SetLocalForceMaxQueryParallelization(void);
extern void SortTupleStore(CitusScanState *scanState);

View File

@ -256,7 +256,7 @@ typedef struct DistributedPlan
/* target list of an INSERT ... SELECT via the coordinator */
List *insertTargetList;
/* target relation of an INSERT ... SELECT via the coordinator */
/* target relation of a modification */
Oid targetRelationId;
/*

View File

@ -54,8 +54,16 @@ extern ShardPlacementAccess * CreatePlacementAccess(ShardPlacement *placement,
ShardPlacementAccessType accessType);
/* helper functions */
extern void AcquireExecutorShardLock(Task *task, CmdType commandType);
extern void AcquireExecutorMultiShardLocks(List *taskList);
extern ShardPlacementAccess * CreatePlacementAccess(ShardPlacement *placement,
ShardPlacementAccessType accessType);
extern bool TaskListRequires2PC(List *taskList);
extern bool ReadOnlyTask(TaskType taskType);
extern List * BuildPlacementSelectList(int32 groupId, List *relationShardList);
extern List * BuildPlacementDDLList(int32 groupId, List *relationShardList);
extern void ExtractParametersFromParamListInfo(ParamListInfo paramListInfo,
Oid **parameterTypes,
const char ***parameterValues);
#endif /* MULTI_ROUTER_EXECUTOR_H_ */

View File

@ -93,10 +93,11 @@ typedef enum
typedef enum
{
MULTI_EXECUTOR_INVALID_FIRST = 0,
MULTI_EXECUTOR_REAL_TIME = 1,
MULTI_EXECUTOR_TASK_TRACKER = 2,
MULTI_EXECUTOR_ROUTER = 3,
MULTI_EXECUTOR_COORDINATOR_INSERT_SELECT = 4
MULTI_EXECUTOR_ADAPTIVE = 1,
MULTI_EXECUTOR_REAL_TIME = 2,
MULTI_EXECUTOR_TASK_TRACKER = 3,
MULTI_EXECUTOR_ROUTER = 4,
MULTI_EXECUTOR_COORDINATOR_INSERT_SELECT = 5
} MultiExecutorType;

View File

@ -45,7 +45,6 @@ extern MultiConnection * GetPlacementConnection(uint32 flags,
extern MultiConnection * StartPlacementConnection(uint32 flags,
struct ShardPlacement *placement,
const char *userName);
extern MultiConnection * GetConnectionIfPlacementAccessedInXact(int flags,
List *placementAccessList,
const char *userName);
@ -57,6 +56,8 @@ extern MultiConnection * StartPlacementListConnection(uint32 flags,
const char *userName);
extern ShardPlacementAccess * CreatePlacementAccess(ShardPlacement *placement,
ShardPlacementAccessType accessType);
extern void AssignPlacementListToConnection(List *placementAccessList,
MultiConnection *connection);
extern void ResetPlacementConnectionManagement(void);
extern void MarkFailedShardPlacements(void);
@ -67,6 +68,7 @@ extern void ResetShardPlacementAssociation(struct MultiConnection *connection);
extern void InitPlacementConnectionManagement(void);
extern bool ConnectionModifiedPlacement(MultiConnection *connection);
extern bool ConnectionUsedForAnyPlacements(MultiConnection *connection);
#endif /* PLACEMENT_CONNECTION_H */

View File

@ -32,6 +32,12 @@ typedef enum
REMOTE_TRANS_STARTING,
REMOTE_TRANS_STARTED,
/* command execution */
REMOTE_TRANS_SENT_BEGIN,
REMOTE_TRANS_SENT_COMMAND,
REMOTE_TRANS_FETCHING_RESULTS,
REMOTE_TRANS_CLEARING_RESULTS,
/* 2pc prepare */
REMOTE_TRANS_PREPARING,
REMOTE_TRANS_PREPARED,
@ -77,6 +83,9 @@ typedef struct RemoteTransaction
/* 2PC transaction name currently associated with connection */
char preparedName[NAMEDATALEN];
/* set when BEGIN is sent over the connection */
bool beginSent;
} RemoteTransaction;

View File

@ -64,6 +64,21 @@ check-multi: all tempinstall-main
$(pg_regress_multi_check) --load-extension=citus \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_schedule $(EXTRA_TESTS)
check-multi-non-adaptive: all tempinstall-main
$(pg_regress_multi_check) --load-extension=citus \
--server-option=citus.task_executor_type=real-time \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_schedule $(EXTRA_TESTS)
check-failure-non-adaptive: all tempinstall-main
$(pg_regress_multi_check) --load-extension=citus --mitmproxy \
--server-option=citus.task_executor_type=real-time \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/failure_schedule $(EXTRA_TESTS)
check-isolation-non-adaptive: all tempinstall-main
$(pg_regress_multi_check) --load-extension=citus --isolationtester \
--server-option=citus.task_executor_type=real-time \
-- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/isolation_schedule $(EXTRA_TESTS)
check-multi-vg: all tempinstall-main
$(pg_regress_multi_check) --load-extension=citus --valgrind \
--pg_ctl-timeout=360 --connection-timeout=500000 --valgrind-path=valgrind --valgrind-log-file=$(VALGRIND_LOG_FILE) \

View File

@ -32,4 +32,27 @@ s/"(target_table_|target_table_|test_ref_table_)[0-9]+"/"\1xxxxxxx"/g
s/\(col_1\)=\([0-9]+\)/(col_1)=(X)/g
# In multi_name_lengths, normalize shard names
s/name_len_12345678901234567890123456789012345678_fcd8ab6f_[0-9+]/name_len_12345678901234567890123456789012345678_fcd8ab6f_xxxxx/g
s/name_len_12345678901234567890123456789012345678_fcd8ab6f_[0-9]+/name_len_12345678901234567890123456789012345678_fcd8ab6f_xxxxx/g
# normalize pkey constraints in multi_insert_select.sql
s/"(raw_events_second_user_id_value_1_key_|agg_events_user_id_value_1_agg_key_)[0-9]+"/"\1xxxxxxx"/g
# normalize explain outputs, basically wipeout the executor name from the output
s/.*Custom Scan \(Citus.*/Custom Scan \(Citus\)/g
s/.*-------------.*/---------------------------------------------------------------------/g
s/.* QUERY PLAN .*/ QUERY PLAN /g
s/.*Custom Plan Provider.*Citus.*/ \"Custom Plan Provider\": \"Citus\", /g
s/.*Custom-Plan-Provide.*/\<Custom-Plan-Provider\>Citus Unified\<\/Custom-Plan-Provider\> /g
s/ +$//g
# normalize shard ids in failure_vaccum
s/10209[0-9] \| 3/10209x \| 3/g
# normalize failed task ids
s/ERROR: failed to execute task [0-9]+/ERROR: failed to execute task X/g
# ignore could not consume warnings
/WARNING: could not consume data from worker node/d
# ignore WAL warnings
/DEBUG: .+creating and filling new WAL file/d

View File

@ -5,7 +5,32 @@ foreign_key_to_reference_table
failure_copy_on_hash
failure_savepoints
foreign_key_restriction_enforcement
failure_real_time_select
failure_vacuum
isolation_citus_dist_activity
multi_insert_select
multi_insert_select_conflict
multi_multiuser
multi_name_lengths
multi_partition_pruning
multi_subtransactions
multi_modifying_xacts
multi_insert_select
# the following tests' output are
# normalized for EXPLAIN outputs
# where the executor name is wiped out
multi_join_order_tpch_small
multi_join_pruning
multi_orderby_limit_pushdown
multi_partition_pruning
multi_select_distinct
multi_subquery_window_functions
multi_task_assignment_policy
multi_view
multi_explain
multi_null_minmax_value_pruning
window_functions
multi_having_pushdown
multi_partitioning
multi_mx_explain

View File

@ -0,0 +1,56 @@
CREATE SCHEMA adaptive_executor;
SET search_path TO adaptive_executor;
CREATE TABLE test (x int, y int);
SET citus.shard_count TO 4;
SET citus.shard_replication_factor TO 1;
SET citus.next_shard_id TO 801009000;
SELECT create_distributed_table('test','x');
create_distributed_table
--------------------------
(1 row)
INSERT INTO test VALUES (1,2);
INSERT INTO test VALUES (3,2);
-- Set a very high slow start to avoid opening parallel connections
SET citus.executor_slow_start_interval TO '1s';
SET citus.max_adaptive_executor_pool_size TO 2;
SET citus.task_executor_type TO 'adaptive';
BEGIN;
SELECT count(*) FROM test a JOIN (SELECT x, pg_sleep(0.1) FROM test) b USING (x);
count
-------
2
(1 row)
SELECT sum(result::bigint) FROM run_command_on_workers($$
SELECT count(*) FROM pg_stat_activity
WHERE pid <> pg_backend_pid() AND query LIKE '%8010090%'
$$);
sum
-----
2
(1 row)
END;
-- SELECT takes longer than slow start interval, should open multiple connections
SET citus.executor_slow_start_interval TO '10ms';
BEGIN;
SELECT count(*) FROM test a JOIN (SELECT x, pg_sleep(0.1) FROM test) b USING (x);
count
-------
2
(1 row)
SELECT sum(result::bigint) FROM run_command_on_workers($$
SELECT count(*) FROM pg_stat_activity
WHERE pid <> pg_backend_pid() AND query LIKE '%8010090%'
$$);
sum
-----
4
(1 row)
END;
DROP SCHEMA adaptive_executor CASCADE;
NOTICE: drop cascades to table test

View File

@ -4,6 +4,8 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row)
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 1;
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- one shard per worker
SET citus.multi_shard_commit_protocol TO '1pc';
@ -159,6 +161,12 @@ SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT count(1) FROM copy_test;
WARNING: could not consume data from worker node
count
@ -167,7 +175,7 @@ WARNING: could not consume data from worker node
(1 row)
---- cancel the connection when we send the data ----
SELECT citus.mitmproxy(format('conn.onCopyData().cancel(%s)', pg_backend_pid()));
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -225,9 +233,9 @@ SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 112
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 110
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 113
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 111
(4 rows)
SELECT count(1) FROM copy_test;
@ -258,8 +266,8 @@ SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 112
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 113
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 110
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 111
(4 rows)
SELECT count(1) FROM copy_test;
@ -278,7 +286,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
@ -288,10 +296,10 @@ SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 112
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 113
copy_test | 100409 | t | 0 | 3 | 100409 | 3 | 8192 | localhost | 9060 | 116
copy_test | 100409 | t | 0 | 3 | 100409 | 1 | 8192 | localhost | 57637 | 117
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 110
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 111
copy_test | 100409 | t | 0 | 3 | 100409 | 3 | 8192 | localhost | 9060 | 114
copy_test | 100409 | t | 0 | 3 | 100409 | 1 | 8192 | localhost | 57637 | 115
(6 rows)
SELECT count(1) FROM copy_test;

View File

@ -0,0 +1,321 @@
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 1;
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- one shard per worker
SET citus.multi_shard_commit_protocol TO '1pc';
SET citus.next_shard_id TO 100400;
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100;
CREATE TABLE copy_test (key int, value int);
SELECT create_distributed_table('copy_test', 'key', 'append');
create_distributed_table
--------------------------
(1 row)
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
SELECT count(1) FROM copy_test;
count
-------
4
(1 row)
SELECT citus.dump_network_traffic();
dump_network_traffic
-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
(0,coordinator,"[initial message]")
(0,worker,"['AuthenticationOk()', 'ParameterStatus(application_name=citus)', 'ParameterStatus(client_encoding=UTF8)', 'ParameterStatus(DateStyle=ISO, MDY)', 'ParameterStatus(integer_datetimes=on)', 'ParameterStatus(IntervalStyle=postgres)', 'ParameterStatus(is_superuser=on)', 'ParameterStatus(server_encoding=UTF8)', 'ParameterStatus(server_version=XXX)', 'ParameterStatus(session_authorization=postgres)', 'ParameterStatus(standard_conforming_strings=on)', 'ParameterStatus(TimeZone=XXX)', 'BackendKeyData(XXX)', 'ReadyForQuery(state=idle)']")
(0,coordinator,"[""Query(query=SELECT worker_apply_shard_ddl_command (100400, 'CREATE TABLE public.copy_test (key integer, value integer)'))""]")
(0,worker,"[""RowDescription(fieldcount=1,fields=['F(name=worker_apply_shard_ddl_command,tableoid=0,colattrnum=0,typoid=2278,typlen=4,typmod=-1,format_code=0)'])"", 'DataRow(columncount=1,columns=[""C(length=0,value=b\\'\\')""])', 'CommandComplete(command=SELECT 1)', 'ReadyForQuery(state=idle)']")
(0,coordinator,"[""Query(query=SELECT worker_apply_shard_ddl_command (100400, 'ALTER TABLE public.copy_test OWNER TO postgres'))""]")
(0,worker,"[""RowDescription(fieldcount=1,fields=['F(name=worker_apply_shard_ddl_command,tableoid=0,colattrnum=0,typoid=2278,typlen=4,typmod=-1,format_code=0)'])"", 'DataRow(columncount=1,columns=[""C(length=0,value=b\\'\\')""])', 'CommandComplete(command=SELECT 1)', 'ReadyForQuery(state=idle)']")
(0,coordinator,"[""Query(query=BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED;SELECT assign_distributed_transaction_id(0, XX, 'XXXX-XX-XX XX:XX:XX.XXXXXX-XX');)""]")
(0,worker,"['CommandComplete(command=BEGIN)', ""RowDescription(fieldcount=1,fields=['F(name=assign_distributed_transaction_id,tableoid=0,colattrnum=0,typoid=2278,typlen=4,typmod=-1,format_code=0)'])"", 'DataRow(columncount=1,columns=[""C(length=0,value=b\\'\\')""])', 'CommandComplete(command=SELECT 1)', 'ReadyForQuery(state=in_transaction_block)']")
(0,coordinator,"['Query(query=COPY public.copy_test_XXXXXX FROM STDIN WITH (FORMAT BINARY))']")
(0,worker,"[""Backend(type=G,body=b'\\\\x01\\\\x00\\\\x02\\\\x00\\\\x01\\\\x00\\\\x01')""]")
(0,coordinator,"[""CopyData(data=b'PGCOPY\\\\n\\\\xff\\\\r\\\\n\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00')"", ""CopyData(data=b'\\\\x00\\\\x02\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x00')"", ""CopyData(data=b'\\\\x00\\\\x02\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x01\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x01')"", ""CopyData(data=b'\\\\x00\\\\x02\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x02\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x04')"", ""CopyData(data=b'\\\\x00\\\\x02\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\x03\\\\x00\\\\x00\\\\x00\\\\x04\\\\x00\\\\x00\\\\x00\\\\t')"", ""CopyData(data=b'\\\\xff\\\\xff')"", 'CopyDone()']")
(0,worker,"['CommandComplete(command=COPY 4)', 'ReadyForQuery(state=in_transaction_block)']")
(0,coordinator,"[""Query(query=SELECT pg_table_size('public.copy_test_XXXXXX'))""]")
(0,worker,"[""RowDescription(fieldcount=1,fields=['F(name=pg_table_size,tableoid=0,colattrnum=0,typoid=20,typlen=8,typmod=-1,format_code=0)'])"", 'DataRow(columncount=1,columns=[""C(length=0,value=b\\'\\')""])', 'CommandComplete(command=SELECT 1)', 'ReadyForQuery(state=in_transaction_block)']")
(0,coordinator,"['Query(query=SELECT min(key), max(key) FROM public.copy_test_XXXXXX)']")
(0,worker,"[""RowDescription(fieldcount=2,fields=['F(name=min,tableoid=0,colattrnum=0,typoid=23,typlen=4,typmod=-1,format_code=0)', 'F(name=max,tableoid=0,colattrnum=0,typoid=23,typlen=4,typmod=-1,format_code=0)'])"", 'DataRow(columncount=2,columns=[""C(length=0,value=b\\'\\')"", ""C(length=1,value=b\\'0\\')""])', 'CommandComplete(command=SELECT 1)', 'ReadyForQuery(state=in_transaction_block)']")
(0,coordinator,"['Query(query=COMMIT)']")
(0,worker,"['CommandComplete(command=COMMIT)', 'ReadyForQuery(state=idle)']")
(0,coordinator,"['Query(query=SELECT count(1) AS count FROM copy_test_100400 copy_test WHERE true)']")
(0,worker,"[""RowDescription(fieldcount=1,fields=['F(name=count,tableoid=0,colattrnum=0,typoid=20,typlen=8,typmod=-1,format_code=0)'])"", 'DataRow(columncount=1,columns=[""C(length=0,value=b\\'\\')""])', 'CommandComplete(command=SELECT 1)', 'ReadyForQuery(state=idle)']")
(20 rows)
---- all of the following tests test behavior with 2 shard placements ----
SHOW citus.shard_replication_factor;
citus.shard_replication_factor
--------------------------------
2
(1 row)
---- kill the connection when we try to create the shard ----
SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT count(1) FROM copy_test;
count
-------
4
(1 row)
---- kill the connection when we try to start a transaction ----
SELECT citus.mitmproxy('conn.onQuery(query="assign_distributed_transaction_id").kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
ERROR: failure on connection marked as essential: localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT count(1) FROM copy_test;
count
-------
4
(1 row)
---- kill the connection when we start the COPY ----
SELECT citus.mitmproxy('conn.onQuery(query="FROM STDIN WITH").kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT count(1) FROM copy_test;
count
-------
4
(1 row)
---- kill the connection when we send the data ----
SELECT citus.mitmproxy('conn.onCopyData().kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
ERROR: failed to COPY to shard 100404 on localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT count(1) FROM copy_test;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
count
-------
4
(1 row)
---- cancel the connection when we send the data ----
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
ERROR: canceling statement due to user request
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT count(1) FROM copy_test;
ERROR: canceling statement due to user request
---- kill the connection when we try to get the size of the table ----
SELECT citus.mitmproxy('conn.onQuery(query="pg_table_size").kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
ERROR: failure on connection marked as essential: localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
(2 rows)
SELECT count(1) FROM copy_test;
count
-------
4
(1 row)
-- we round-robin when picking which node to run pg_table_size on, this COPY runs it on
-- the other node, so the next copy will try to run it on our node
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY p.nodeport, p.placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 110
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 111
(4 rows)
SELECT count(1) FROM copy_test;
count
-------
8
(1 row)
---- kill the connection when we try to get the min, max of the table ----
SELECT citus.mitmproxy('conn.onQuery(query="SELECT min\(key\), max\(key\)").kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
ERROR: failure on connection marked as essential: localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 110
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 111
(4 rows)
SELECT count(1) FROM copy_test;
count
-------
8
(1 row)
---- kill the connection when we try to COMMIT ----
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
mitmproxy
-----------
(1 row)
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
SELECT * FROM pg_dist_shard s, pg_dist_shard_placement p
WHERE (s.shardid = p.shardid) AND s.logicalrelid = 'copy_test'::regclass
ORDER BY placementid;
logicalrelid | shardid | shardstorage | shardminvalue | shardmaxvalue | shardid | shardstate | shardlength | nodename | nodeport | placementid
--------------+---------+--------------+---------------+---------------+---------+------------+-------------+-----------+----------+-------------
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 57637 | 100
copy_test | 100400 | t | 0 | 3 | 100400 | 1 | 8192 | localhost | 9060 | 101
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 9060 | 110
copy_test | 100407 | t | 0 | 3 | 100407 | 1 | 8192 | localhost | 57637 | 111
copy_test | 100409 | t | 0 | 3 | 100409 | 3 | 8192 | localhost | 9060 | 114
copy_test | 100409 | t | 0 | 3 | 100409 | 1 | 8192 | localhost | 57637 | 115
(6 rows)
SELECT count(1) FROM copy_test;
count
-------
12
(1 row)
-- ==== Clean up, we're done here ====
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE copy_test;

View File

@ -4,6 +4,8 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row)
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 0;
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- one shard per worker
SET citus.multi_shard_commit_protocol TO '1pc';
@ -124,7 +126,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT$").killall()');
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
-- the shard is marked invalid
@ -325,7 +327,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT$").killall()');
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: could not commit transaction for shard 100400 on any active node
@ -365,7 +367,7 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="COMMIT").killall()');
COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' WITH CSV;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: could not commit transaction for shard 100400 on any active node

View File

@ -81,13 +81,13 @@ SELECT citus.mitmproxy('conn.delay(500)');
-- we cannot control which replica of the reference table will be queried and there is
-- only one specific client we can control the connection for.
-- by using round-robin task_assignment_policy we can force to hit both machines. We will
-- use two output files to match both orders to verify there is 1 that times out and falls
-- through to read from the other machine
-- by using round-robin task_assignment_policy we can force to hit both machines.
-- and in the end, dumping the network traffic shows that the connection establishment
-- is initiated to the node behind the proxy
SET client_min_messages TO ERROR;
SET citus.task_assignment_policy TO 'round-robin';
-- suppress the warning since we can't control which shard is chose first. Failure of this
-- test would be if one of the queries does not return the result but an error.
SET client_min_messages TO ERROR;
SELECT name FROM r1 WHERE id = 2;
name
------
@ -108,6 +108,119 @@ SELECT citus.dump_network_traffic();
(0,coordinator,"[initial message]")
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- similar test with the above but this time on a
-- distributed table instead of a reference table
-- and with citus.force_max_query_parallelization is set
SET citus.force_max_query_parallelization TO ON;
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
-- suppress the warning since we can't control which shard is chose first. Failure of this
-- test would be if one of the queries does not return the result but an error.
SELECT count(*) FROM products;
count
-------
0
(1 row)
SELECT count(*) FROM products;
count
-------
0
(1 row)
-- use OFFSET 1 to prevent printing the line where source
-- is the worker
SELECT citus.dump_network_traffic() ORDER BY 1 OFFSET 1;
dump_network_traffic
-------------------------------------
(1,coordinator,"[initial message]")
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_replication_factor TO 1;
CREATE TABLE single_replicatated(key int);
SELECT create_distributed_table('single_replicatated', 'key');
create_distributed_table
--------------------------
(1 row)
-- this time the table is single replicated and we're still using the
-- the max parallelization flag, so the query should fail
SET citus.force_max_query_parallelization TO ON;
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM single_replicatated;
ERROR: failed to execute task 1
SET citus.force_max_query_parallelization TO OFF;
-- one similar test, but this time on modification queries
-- to see that connection establishement failures could
-- mark placement INVALID
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT
count(*) as invalid_placement_count
FROM
pg_dist_shard_placement
WHERE
shardstate = 3 AND
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
invalid_placement_count
-------------------------
0
(1 row)
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
INSERT INTO products VALUES (100, '100', 100);
COMMIT;
SELECT
count(*) as invalid_placement_count
FROM
pg_dist_shard_placement
WHERE
shardstate = 3 AND
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
invalid_placement_count
-------------------------
1
(1 row)
-- show that INSERT went through
SELECT count(*) FROM products WHERE product_no = 100;
count
-------
1
(1 row)
RESET client_min_messages;
-- verify get_global_active_transactions works when a timeout happens on a connection
SELECT get_global_active_transactions();
@ -123,8 +236,10 @@ SELECT citus.mitmproxy('conn.allow()');
(1 row)
SET citus.node_connection_timeout TO DEFAULT;
DROP SCHEMA fail_connect CASCADE;
NOTICE: drop cascades to 2 other objects
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table products
drop cascades to table r1
drop cascades to table single_replicatated
SET search_path TO default;

View File

@ -0,0 +1,244 @@
--
-- failure_connection_establishment.sql tests some behaviour of connection management when
-- it fails to connect.
--
-- Failure cases covered:
-- - timeout
--
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE SCHEMA fail_connect;
SET search_path TO 'fail_connect';
SET citus.shard_count TO 4;
SET citus.max_cached_conns_per_worker TO 0;
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1450000;
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 1450000;
CREATE TABLE products (
product_no integer,
name text,
price numeric
);
SELECT create_distributed_table('products', 'product_no');
create_distributed_table
--------------------------
(1 row)
-- Can only add primary key constraint on distribution column (or group of columns
-- including distribution column)
-- Command below should error out since 'name' is not a distribution column
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(name);
ERROR: cannot create constraint on "products"
DETAIL: Distributed relations cannot have UNIQUE, EXCLUDE, or PRIMARY KEY constraints that do not include the partition column (with an equality operator if EXCLUDE).
-- we will insert a connection delay here as this query was the cause for an investigation
-- into connection establishment problems
SET citus.node_connection_timeout TO 400;
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
ALTER TABLE products ADD CONSTRAINT p_key PRIMARY KEY(product_no);
ERROR: could not establish any connections to the node localhost:9060 after 400 ms
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE r1 (
id int PRIMARY KEY,
name text
);
INSERT INTO r1 (id, name) VALUES
(1,'foo'),
(2,'bar'),
(3,'baz');
SELECT create_reference_table('r1');
NOTICE: Copying data from local table...
create_reference_table
------------------------
(1 row)
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
-- we cannot control which replica of the reference table will be queried and there is
-- only one specific client we can control the connection for.
-- by using round-robin task_assignment_policy we can force to hit both machines.
-- and in the end, dumping the network traffic shows that the connection establishment
-- is initiated to the node behind the proxy
SET client_min_messages TO ERROR;
SET citus.task_assignment_policy TO 'round-robin';
-- suppress the warning since we can't control which shard is chose first. Failure of this
-- test would be if one of the queries does not return the result but an error.
SELECT name FROM r1 WHERE id = 2;
name
------
bar
(1 row)
SELECT name FROM r1 WHERE id = 2;
name
------
bar
(1 row)
-- verify a connection attempt was made to the intercepted node, this would have cause the
-- connection to have been delayed and thus caused a timeout
SELECT citus.dump_network_traffic();
dump_network_traffic
-------------------------------------
(0,coordinator,"[initial message]")
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- similar test with the above but this time on a
-- distributed table instead of a reference table
-- and with citus.force_max_query_parallelization is set
SET citus.force_max_query_parallelization TO ON;
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
-- suppress the warning since we can't control which shard is chose first. Failure of this
-- test would be if one of the queries does not return the result but an error.
SELECT count(*) FROM products;
count
-------
0
(1 row)
SELECT count(*) FROM products;
count
-------
0
(1 row)
-- use OFFSET 1 to prevent printing the line where source
-- is the worker
SELECT citus.dump_network_traffic() ORDER BY 1 OFFSET 1;
dump_network_traffic
-------------------------------------
(1,coordinator,"[initial message]")
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_replication_factor TO 1;
CREATE TABLE single_replicatated(key int);
SELECT create_distributed_table('single_replicatated', 'key');
create_distributed_table
--------------------------
(1 row)
-- this time the table is single replicated and we're still using the
-- the max parallelization flag, so the query should fail
SET citus.force_max_query_parallelization TO ON;
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM single_replicatated;
ERROR: could not establish any connections to the node localhost:9060 after 400 ms
SET citus.force_max_query_parallelization TO OFF;
-- one similar test, but this time on modification queries
-- to see that connection establishement failures could
-- mark placement INVALID
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT
count(*) as invalid_placement_count
FROM
pg_dist_shard_placement
WHERE
shardstate = 3 AND
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
invalid_placement_count
-------------------------
0
(1 row)
SELECT citus.mitmproxy('conn.delay(500)');
mitmproxy
-----------
(1 row)
INSERT INTO products VALUES (100, '100', 100);
COMMIT;
SELECT
count(*) as invalid_placement_count
FROM
pg_dist_shard_placement
WHERE
shardstate = 3 AND
shardid IN (SELECT shardid from pg_dist_shard where logicalrelid = 'products'::regclass);
invalid_placement_count
-------------------------
1
(1 row)
-- show that INSERT went through
SELECT count(*) FROM products WHERE product_no = 100;
count
-------
1
(1 row)
RESET client_min_messages;
-- verify get_global_active_transactions works when a timeout happens on a connection
SELECT get_global_active_transactions();
WARNING: could not establish connection after 400 ms
WARNING: connection error: localhost:9060
get_global_active_transactions
--------------------------------
(0 rows)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.node_connection_timeout TO DEFAULT;
DROP SCHEMA fail_connect CASCADE;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table products
drop cascades to table r1
drop cascades to table single_replicatated
SET search_path TO default;

View File

@ -1,6 +1,9 @@
--
-- Failure tests for COPY to reference tables
--
-- We have to keep two copies of this failure test
-- because if the shards are created via the executor
-- cancellations are processed, otherwise they are not
CREATE SCHEMA create_distributed_table_non_empty_failure;
SET search_path TO 'create_distributed_table_non_empty_failure';
SET citus.next_shard_id TO 11000000;
@ -137,7 +140,8 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata
(2 rows)
-- cancel as soon as the coordinator sends begin
-- shards will be created because we ignore cancel requests during the shard creation
-- if the shards are created via the executor, the table creation will fail
-- otherwise shards will be created because we ignore cancel requests during the shard creation
-- Interrupts are hold in CreateShardsWithRoundRobinPolicy
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
@ -659,7 +663,8 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata
(2 rows)
-- cancel as soon as the coordinator sends begin
-- shards will be created because we ignore cancel requests during the shard creation
-- if the shards are created via the executor, the table creation will fail
-- otherwise shards will be created because we ignore cancel requests during the shard creation
-- Interrupts are hold in CreateShardsWithRoundRobinPolicy
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy

View File

@ -0,0 +1,979 @@
--
-- Failure tests for COPY to reference tables
--
-- We have to keep two copies of this failure test
-- because if the shards are created via the executor
-- cancellations are processed, otherwise they are not
CREATE SCHEMA create_distributed_table_non_empty_failure;
SET search_path TO 'create_distributed_table_non_empty_failure';
SET citus.next_shard_id TO 11000000;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- we'll start with replication factor 1 and 2pc
SET citus.shard_replication_factor TO 1;
SET citus.shard_count to 4;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
-- in the first test, kill the first connection we sent from the coordinator
SELECT citus.mitmproxy('conn.kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- in the first test, cancel the first connection we sent from the coordinator
SELECT citus.mitmproxy('conn.cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill as soon as the coordinator sends CREATE SCHEMA
SELECT citus.mitmproxy('conn.onQuery(query="^CREATE SCHEMA").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,1)
(2 rows)
-- cancel as soon as the coordinator sends CREATE SCHEMA
-- Note: Schema should be created in workers because Citus
-- does not check for interrupts until GetRemoteCommandResult is called.
-- Since we already sent the command at this stage, the schemas get created in workers
SELECT citus.mitmproxy('conn.onQuery(query="^CREATE SCHEMA").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers
------------------------
(localhost,9060,t,1)
(localhost,57637,t,1)
(2 rows)
SELECT run_command_on_workers($$DROP SCHEMA IF EXISTS create_distributed_table_non_empty_failure$$);
run_command_on_workers
-----------------------------------
(localhost,9060,t,"DROP SCHEMA")
(localhost,57637,t,"DROP SCHEMA")
(2 rows)
-- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers
------------------------
(localhost,9060,t,1)
(localhost,57637,t,1)
(2 rows)
-- cancel as soon as the coordinator sends begin
-- if the shards are created via the executor, the table creation will fail
-- otherwise shards will be created because we ignore cancel requests during the shard creation
-- Interrupts are hold in CreateShardsWithRoundRobinPolicy
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers
------------------------
(localhost,9060,t,1)
(localhost,57637,t,1)
(2 rows)
DROP TABLE test_table ;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
-- kill as soon as the coordinator sends CREATE TABLE
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill as soon as the coordinator sends COPY
SELECT citus.mitmproxy('conn.onQuery(query="COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill when the COPY is completed, it should be rollbacked properly
SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
NOTICE: Copying data from local table...
ERROR: failed to COPY to shard 11000016 on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- cancel as soon as the coordinator sends COPY, table
-- should not be created and rollbacked properly
SELECT citus.mitmproxy('conn.onQuery(query="COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- cancel when the COPY is completed, it should be rollbacked properly
SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
NOTICE: Copying data from local table...
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- immediately kill when we see prepare transaction to see if the command
-- successfully rollbacked the created shards
-- we don't want to see the prepared transaction numbers in the warnings
SET client_min_messages TO ERROR;
SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- immediately cancel when we see prepare transaction to see if the command
-- successfully rollbacked the created shards
SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
1
(1 row)
-- kill as soon as the coordinator sends COMMIT
-- shards should be created and kill should not affect
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT PREPARED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
4
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
2
(1 row)
DROP TABLE test_table ;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
-- cancel as soon as the coordinator sends COMMIT
-- shards should be created and kill should not affect
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT PREPARED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
4
(1 row)
DROP TABLE test_table ;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
-- kill as soon as the coordinator sends ROLLBACK
-- the command can be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
ROLLBACK;
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- cancel as soon as the coordinator sends ROLLBACK
-- should be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- We are done with pure create_distributed_table testing and now
-- testing for co-located tables.
CREATE TABLE colocated_table(id int, value_1 int);
SELECT create_distributed_table('colocated_table', 'id');
create_distributed_table
--------------------------
(1 row)
-- Now, cancel the connection just after transaction is opened on
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- Now, kill the connection just after transaction is opened on
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, cancel the connection just after the COPY started to
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- Now, kill the connection just after the COPY started to
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, cancel the connection when we issue CREATE TABLE on
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_apply_shard_ddl_command").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- Now, kill the connection when we issue CREATE TABLE on
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_apply_shard_ddl_command").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now run the same tests with 1pc
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE colocated_table;
DROP TABLE test_table;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
SET citus.multi_shard_commit_protocol TO '1pc';
SELECT citus.mitmproxy('conn.kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- in the first test, cancel the first connection we sent from the coordinator
SELECT citus.mitmproxy('conn.cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- kill as soon as the coordinator sends begin
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers
------------------------
(localhost,9060,t,1)
(localhost,57637,t,1)
(2 rows)
-- cancel as soon as the coordinator sends begin
-- if the shards are created via the executor, the table creation will fail
-- otherwise shards will be created because we ignore cancel requests during the shard creation
-- Interrupts are hold in CreateShardsWithRoundRobinPolicy
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$);
run_command_on_workers
------------------------
(localhost,9060,t,1)
(localhost,57637,t,1)
(2 rows)
DROP TABLE test_table ;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
-- kill as soon as the coordinator sends CREATE TABLE
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill as soon as the coordinator sends COPY
SELECT citus.mitmproxy('conn.onQuery(query="COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill when the COPY is completed, it should be rollbacked properly
SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: failed to COPY to shard 11000096 on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- cancel as soon as the coordinator sends COPY, table
-- should not be created and rollbacked properly
SELECT citus.mitmproxy('conn.onQuery(query="COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- cancel when the COPY is completed, it should be rollbacked properly
SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill as soon as the coordinator sends ROLLBACK
-- the command can be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
ROLLBACK;
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- cancel as soon as the coordinator sends ROLLBACK
-- should be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- kill as soon as the coordinator sends COMMIT
-- the command can be COMMITed
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
COMMIT;
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
4
(1 row)
DROP TABLE test_table;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
-- cancel as soon as the coordinator sends COMMIT
-- should be COMMITed
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table', 'id');
create_distributed_table
--------------------------
(1 row)
COMMIT;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
4
(1 row)
DROP TABLE test_table;
CREATE TABLE test_table(id int, value_1 int);
INSERT INTO test_table VALUES (1,1),(2,2),(3,3),(4,4);
CREATE TABLE colocated_table(id int, value_1 int);
SELECT create_distributed_table('colocated_table', 'id');
create_distributed_table
--------------------------
(1 row)
-- Now, cancel the connection just after transaction is opened on
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- Now, kill the connection just after transaction is opened on
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- Now, cancel the connection just after the COPY started to
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
-- Now, kill the connection just after the COPY started to
-- workers. Note that, when there is a colocated table, interrupts
-- are not held and we can cancel in the middle of the execution
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id', colocate_with => 'colocated_table');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_non_empty_failure.test_table'::regclass;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'create_distributed_table_non_empty_failure' and table_name LIKE 'test_table%'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP SCHEMA create_distributed_table_non_empty_failure CASCADE;

View File

@ -0,0 +1,248 @@
--
-- Failure tests for creating reference table
--
CREATE SCHEMA failure_reference_table;
SET search_path TO 'failure_reference_table';
SET citus.next_shard_id TO 10000000;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE ref_table(id int);
INSERT INTO ref_table VALUES(1),(2),(3);
-- Kill on sending first query to worker node, should error
-- out and not create any placement
SELECT citus.mitmproxy('conn.onQuery().kill()');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- Kill after creating transaction on worker node
SELECT citus.mitmproxy('conn.onCommandComplete(command="BEGIN").kill()');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- Cancel after creating transaction on worker node
SELECT citus.mitmproxy('conn.onCommandComplete(command="BEGIN").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- Kill after copying data to worker node
SELECT citus.mitmproxy('conn.onCommandComplete(command="SELECT 1").kill()');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- Cancel after copying data to worker node
SELECT citus.mitmproxy('conn.onCommandComplete(command="SELECT 1").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- Kill after copying data to worker node
SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY 3").kill()');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
NOTICE: Copying data from local table...
ERROR: failed to COPY to shard 10000004 on localhost:9060
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- Cancel after copying data to worker node
SELECT citus.mitmproxy('conn.onCommandComplete(command="COPY 3").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
NOTICE: Copying data from local table...
ERROR: canceling statement due to user request
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
-- we don't want to see the prepared transaction numbers in the warnings
SET client_min_messages TO ERROR;
-- Kill after preparing transaction. Since we don't commit after preparing, we recover
-- prepared transaction afterwards.
SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").kill()');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
SELECT count(*) FROM pg_dist_shard_placement;
count
-------
0
(1 row)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
1
(1 row)
-- Kill after commiting prepared, this should succeed
SELECT citus.mitmproxy('conn.onCommandComplete(command="COMMIT PREPARED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_reference_table('ref_table');
create_reference_table
------------------------
(1 row)
SELECT shardid, nodeport, shardstate FROM pg_dist_shard_placement ORDER BY shardid, nodeport;
shardid | nodeport | shardstate
----------+----------+------------
10000007 | 9060 | 1
10000007 | 57637 | 1
(2 rows)
SET client_min_messages TO NOTICE;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE ref_table;
CREATE TABLE ref_table(id int);
INSERT INTO ref_table VALUES(1),(2),(3);
-- Test in transaction
SELECT citus.mitmproxy('conn.onQuery().kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_reference_table('ref_table');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
COMMIT;
-- kill on ROLLBACK, should be rollbacked
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_reference_table('ref_table');
NOTICE: Copying data from local table...
create_reference_table
------------------------
(1 row)
ROLLBACK;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
SELECT * FROM pg_dist_shard_placement ORDER BY shardid, nodeport;
shardid | shardstate | shardlength | nodename | nodeport | placementid
---------+------------+-------------+----------+----------+-------------
(0 rows)
-- cancel when the coordinator send ROLLBACK, should be rollbacked. We ignore cancellations
-- during the ROLLBACK.
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_reference_table('ref_table');
NOTICE: Copying data from local table...
create_reference_table
------------------------
(1 row)
ROLLBACK;
SELECT * FROM pg_dist_shard_placement ORDER BY shardid, nodeport;
shardid | shardstate | shardlength | nodename | nodeport | placementid
---------+------------+-------------+----------+----------+-------------
(0 rows)
DROP SCHEMA failure_reference_table CASCADE;
NOTICE: drop cascades to table ref_table
SET search_path TO default;

View File

@ -109,7 +109,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
(2 rows)
-- Now, kill the connection after sending create table command with worker_apply_shard_ddl_command UDF
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_apply_shard_ddl_command").after(2).kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_shard_ddl_command").after(1).kill()');
mitmproxy
-----------
@ -143,7 +143,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
-- with worker_apply_shard_ddl_command UDF.
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT worker_apply_shard_ddl_command").after(2).kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_shard_ddl_command").after(1).kill()');
mitmproxy
-----------
@ -402,7 +402,8 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
-- Now, cancel the connection while creating the transaction on
-- workers. Note that, cancel requests will be ignored during
-- shard creation again in transaction.
-- shard creation again in transaction if we're not relying on the
-- executor. So, we'll have two output files
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -418,6 +419,12 @@ WARNING: cancel requests are ignored during shard creation
(1 row)
COMMIT;
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
1
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
@ -542,7 +549,7 @@ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables W
-- Now, cancel the connection while creating transactions on
-- workers with 1pc. Note that, cancel requests will be ignored during
-- shard creation.
-- shard creation unless the executor is used. So, we'll have two output files
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -558,6 +565,12 @@ WARNING: cancel requests are ignored during shard creation
(1 row)
COMMIT;
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
0
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
@ -666,7 +679,7 @@ ERROR: canceling statement due to user request
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
2
1
(1 row)
SELECT citus.mitmproxy('conn.allow()');

View File

@ -0,0 +1,691 @@
--
-- failure_create_table adds failure tests for creating table without data.
--
CREATE SCHEMA failure_create_table;
SET search_path TO 'failure_create_table';
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_replication_factor TO 1;
SET citus.shard_count to 4;
CREATE TABLE test_table(id int, value_1 int);
-- Kill connection before sending query to the worker
SELECT citus.mitmproxy('conn.kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- kill as soon as the coordinator sends CREATE SCHEMA
-- Since schemas are created in separate transaction, schema will
-- be created only on the node which is not behind the proxy.
-- https://github.com/citusdata/citus/pull/1652
SELECT citus.mitmproxy('conn.onQuery(query="^CREATE SCHEMA").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'failure_create_table'$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,1)
(2 rows)
-- Now, kill the connection while opening transaction on workers.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, kill the connection after sending create table command with worker_apply_shard_ddl_command UDF
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_shard_ddl_command").after(1).kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Kill the connection while creating a distributed table in sequential mode on sending create command
-- with worker_apply_shard_ddl_command UDF.
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
SELECT citus.mitmproxy('conn.onQuery(query="SELECT worker_apply_shard_ddl_command").after(1).kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table', 'id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
COMMIT;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, cancel the connection while creating transaction
-- workers. Note that, cancel requests will be ignored during
-- shard creation.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id');
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
DROP TABLE test_table;
CREATE TABLE test_table(id int, value_1 int);
-- Kill and cancel the connection with colocate_with option while sending the create table command
CREATE TABLE temp_table(id int, value_1 int);
SELECT create_distributed_table('temp_table','id');
create_distributed_table
--------------------------
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id',colocate_with=>'temp_table');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
4
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id',colocate_with=>'temp_table');
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
4
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Kill and cancel the connection after worker sends "PREPARE TRANSACTION" ack with colocate_with option
SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").kill()');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id',colocate_with=>'temp_table');
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
4
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT create_distributed_table('test_table','id',colocate_with=>'temp_table');
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
4
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
DROP TABLE temp_table;
-- Test inside transaction
-- Kill connection before sending query to the worker
SELECT citus.mitmproxy('conn.kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, kill the connection while creating transaction on workers in transaction.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, cancel the connection while creating the transaction on
-- workers. Note that, cancel requests will be ignored during
-- shard creation again in transaction if we're not relying on the
-- executor. So, we'll have two output files
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: canceling statement due to user request
COMMIT;
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
1
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
DROP TABLE test_table;
CREATE TABLE test_table(id int, value_1 int);
-- Test inside transaction and with 1PC
SET citus.multi_shard_commit_protocol TO "1pc";
-- Kill connection before sending query to the worker with 1pc.
SELECT citus.mitmproxy('conn.kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Kill connection while sending create table command with 1pc.
SELECT citus.mitmproxy('conn.onQuery(query="CREATE TABLE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, kill the connection while opening transactions on workers with 1pc. Transaction will be opened due to BEGIN.
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
ROLLBACK;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Now, cancel the connection while creating transactions on
-- workers with 1pc. Note that, cancel requests will be ignored during
-- shard creation unless the executor is used. So, we'll have two output files
SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT create_distributed_table('test_table','id');
ERROR: canceling statement due to user request
COMMIT;
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
0
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
DROP TABLE test_table;
-- Test master_create_worker_shards with 2pc
SET citus.multi_shard_commit_protocol TO "2pc";
CREATE TABLE test_table_2(id int, value_1 int);
SELECT master_create_distributed_table('test_table_2', 'id', 'hash');
master_create_distributed_table
---------------------------------
(1 row)
-- Kill connection before sending query to the worker
SELECT citus.mitmproxy('conn.kill()');
mitmproxy
-----------
(1 row)
SELECT master_create_worker_shards('test_table_2', 4, 2);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Kill the connection after worker sends "PREPARE TRANSACTION" ack
SELECT citus.mitmproxy('conn.onCommandComplete(command="^PREPARE TRANSACTION").kill()');
mitmproxy
-----------
(1 row)
SELECT master_create_worker_shards('test_table_2', 4, 2);
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
-- Cancel the connection after sending prepare transaction in master_create_worker_shards
SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT master_create_worker_shards('test_table_2', 4, 2);
ERROR: canceling statement due to user request
-- Show that there is no pending transaction
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
1
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM pg_dist_shard;
count
-------
0
(1 row)
SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.tables WHERE table_schema = 'failure_create_table' and table_name LIKE 'test_table%' ORDER BY 1$$);
run_command_on_workers
------------------------
(localhost,9060,t,0)
(localhost,57637,t,0)
(2 rows)
DROP SCHEMA failure_create_table CASCADE;
NOTICE: drop cascades to table test_table_2
SET search_path TO default;

View File

@ -0,0 +1,393 @@
CREATE SCHEMA cte_failure;
SET SEARCH_PATH=cte_failure;
SET citus.shard_count to 2;
SET citus.shard_replication_factor to 1;
SET citus.next_shard_id TO 16000000;
SELECT pg_backend_pid() as pid \gset
CREATE TABLE users_table (user_id int, user_name text);
CREATE TABLE events_table(user_id int, event_id int, event_type int);
SELECT create_distributed_table('users_table', 'user_id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_table('events_table', 'user_id');
create_distributed_table
--------------------------
(1 row)
CREATE TABLE users_table_local AS SELECT * FROM users_table;
-- kill at the first copy (push)
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
WITH cte AS (
WITH local_cte AS (
SELECT * FROM users_table_local
),
dist_cte AS (
SELECT user_id FROM events_table
)
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
)
SELECT
count(*)
FROM
cte,
(SELECT
DISTINCT users_table.user_id
FROM
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND
event_type IN (1,2,3,4)
ORDER BY 1 DESC LIMIT 5
) as foo
WHERE foo.user_id = cte.user_id;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- kill at the second copy (pull)
SELECT citus.mitmproxy('conn.onQuery(query="SELECT user_id FROM cte_failure.events_table_16000002").kill()');
mitmproxy
-----------
(1 row)
WITH cte AS (
WITH local_cte AS (
SELECT * FROM users_table_local
),
dist_cte AS (
SELECT user_id FROM events_table
)
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
)
SELECT
count(*)
FROM
cte,
(SELECT
DISTINCT users_table.user_id
FROM
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND
event_type IN (1,2,3,4)
ORDER BY 1 DESC LIMIT 5
) as foo
WHERE foo.user_id = cte.user_id;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- kill at the third copy (pull)
SELECT citus.mitmproxy('conn.onQuery(query="SELECT DISTINCT users_table.user").kill()');
mitmproxy
-----------
(1 row)
WITH cte AS (
WITH local_cte AS (
SELECT * FROM users_table_local
),
dist_cte AS (
SELECT user_id FROM events_table
)
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
)
SELECT
count(*)
FROM
cte,
(SELECT
DISTINCT users_table.user_id
FROM
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND
event_type IN (1,2,3,4)
ORDER BY 1 DESC LIMIT 5
) as foo
WHERE foo.user_id = cte.user_id;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- cancel at the first copy (push)
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
WITH cte AS (
WITH local_cte AS (
SELECT * FROM users_table_local
),
dist_cte AS (
SELECT user_id FROM events_table
)
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
)
SELECT
count(*)
FROM
cte,
(SELECT
DISTINCT users_table.user_id
FROM
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND
event_type IN (1,2,3,4)
ORDER BY 1 DESC LIMIT 5
) as foo
WHERE foo.user_id = cte.user_id;
ERROR: canceling statement due to user request
-- cancel at the second copy (pull)
SELECT citus.mitmproxy('conn.onQuery(query="SELECT user_id FROM").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
WITH cte AS (
WITH local_cte AS (
SELECT * FROM users_table_local
),
dist_cte AS (
SELECT user_id FROM events_table
)
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
)
SELECT
count(*)
FROM
cte,
(SELECT
DISTINCT users_table.user_id
FROM
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND
event_type IN (1,2,3,4)
ORDER BY 1 DESC LIMIT 5
) as foo
WHERE foo.user_id = cte.user_id;
ERROR: canceling statement due to user request
-- cancel at the third copy (pull)
SELECT citus.mitmproxy('conn.onQuery(query="SELECT DISTINCT users_table.user").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
WITH cte AS (
WITH local_cte AS (
SELECT * FROM users_table_local
),
dist_cte AS (
SELECT user_id FROM events_table
)
SELECT dist_cte.user_id FROM local_cte join dist_cte on dist_cte.user_id=local_cte.user_id
)
SELECT
count(*)
FROM
cte,
(SELECT
DISTINCT users_table.user_id
FROM
users_table, events_table
WHERE
users_table.user_id = events_table.user_id AND
event_type IN (1,2,3,4)
ORDER BY 1 DESC LIMIT 5
) as foo
WHERE foo.user_id = cte.user_id;
ERROR: canceling statement due to user request
-- distributed update tests
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- insert some rows
INSERT INTO users_table VALUES (1, 'A'), (2, 'B'), (3, 'C'), (4, 'D'), (5, 'E');
INSERT INTO events_table VALUES (1,1,1), (1,2,1), (1,3,1), (2,1, 4), (3, 4,1), (5, 1, 2), (5, 2, 1), (5, 2,2);
SELECT * FROM users_table ORDER BY 1, 2;
user_id | user_name
---------+-----------
1 | A
2 | B
3 | C
4 | D
5 | E
(5 rows)
-- following will delete and insert the same rows
WITH cte_delete as (DELETE FROM users_table WHERE user_name in ('A', 'D') RETURNING *)
INSERT INTO users_table SELECT * FROM cte_delete;
-- verify contents are the same
SELECT * FROM users_table ORDER BY 1, 2;
user_id | user_name
---------+-----------
1 | A
2 | B
3 | C
4 | D
5 | E
(5 rows)
-- kill connection during deletion
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM").kill()');
mitmproxy
-----------
(1 row)
WITH cte_delete as (DELETE FROM users_table WHERE user_name in ('A', 'D') RETURNING *)
INSERT INTO users_table SELECT * FROM cte_delete;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify contents are the same
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT * FROM users_table ORDER BY 1, 2;
user_id | user_name
---------+-----------
1 | A
2 | B
3 | C
4 | D
5 | E
(5 rows)
-- kill connection during insert
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
WITH cte_delete as (DELETE FROM users_table WHERE user_name in ('A', 'D') RETURNING *)
INSERT INTO users_table SELECT * FROM cte_delete;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- verify contents are the same
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT * FROM users_table ORDER BY 1, 2;
user_id | user_name
---------+-----------
1 | A
2 | B
3 | C
4 | D
5 | E
(5 rows)
-- cancel during deletion
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
WITH cte_delete as (DELETE FROM users_table WHERE user_name in ('A', 'D') RETURNING *)
INSERT INTO users_table SELECT * FROM cte_delete;
ERROR: canceling statement due to user request
-- verify contents are the same
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT * FROM users_table ORDER BY 1, 2;
user_id | user_name
---------+-----------
1 | A
2 | B
3 | C
4 | D
5 | E
(5 rows)
-- cancel during insert
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
WITH cte_delete as (DELETE FROM users_table WHERE user_name in ('A', 'D') RETURNING *)
INSERT INTO users_table SELECT * FROM cte_delete;
ERROR: canceling statement due to user request
-- verify contents are the same
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT * FROM users_table ORDER BY 1, 2;
user_id | user_name
---------+-----------
1 | A
2 | B
3 | C
4 | D
5 | E
(5 rows)
-- test sequential delete/insert
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE FROM").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SET LOCAL citus.multi_shard_modify_mode = 'sequential';
WITH cte_delete as (DELETE FROM users_table WHERE user_name in ('A', 'D') RETURNING *)
INSERT INTO users_table SELECT * FROM cte_delete;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
END;
RESET SEARCH_PATH;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP SCHEMA cte_failure CASCADE;
NOTICE: drop cascades to 3 other objects
DETAIL: drop cascades to table cte_failure.users_table
drop cascades to table cte_failure.events_table
drop cascades to table cte_failure.users_table_local

View File

@ -5,6 +5,8 @@
--
CREATE SCHEMA ddl_failure;
SET search_path TO 'ddl_failure';
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 0;
-- we don't want to see the prepared transaction numbers in the warnings
SET client_min_messages TO ERROR;
SELECT citus.mitmproxy('conn.allow()');
@ -218,12 +220,12 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT").kill()');
ALTER TABLE test_table ADD COLUMN new_column INT;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: could not commit transaction for shard 100802 on any active node

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,150 @@
--
-- failure_insert_select_pushdown
--
-- performs failure/cancellation test for insert/select pushed down to shards.
--
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE SCHEMA insert_select_pushdown;
SET SEARCH_PATH=insert_select_pushdown;
SET citus.shard_count to 2;
SET citus.shard_replication_factor to 1;
SELECT pg_backend_pid() as pid \gset
CREATE TABLE events_table(user_id int, event_id int, event_type int);
CREATE TABLE events_summary(user_id int, event_id int, event_count int);
SELECT create_distributed_table('events_table', 'user_id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_table('events_summary', 'user_id');
create_distributed_table
--------------------------
(1 row)
INSERT INTO events_table VALUES (1, 1, 3 ), (1, 2, 1), (1, 3, 2), (2, 4, 3), (3, 5, 1), (4, 7, 1), (4, 1, 9), (4, 3, 2);
SELECT count(*) FROM events_summary;
count
-------
0
(1 row)
-- insert/select from one distributed table to another
-- kill worker query
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT INTO insert_select_pushdown").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_summary SELECT user_id, event_id, count(*) FROM events_table GROUP BY 1,2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_summary;
count
-------
0
(1 row)
-- cancel worker query
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT INTO insert_select_pushdown").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_summary SELECT user_id, event_id, count(*) FROM events_table GROUP BY 1,2;
ERROR: canceling statement due to user request
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_summary;
count
-------
0
(1 row)
-- test self insert/select
SELECT count(*) FROM events_table;
count
-------
8
(1 row)
-- kill worker query
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT INTO insert_select_pushdown").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_table SELECT * FROM events_table;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_table;
count
-------
8
(1 row)
-- cancel worker query
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT INTO insert_select_pushdown").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_table SELECT * FROM events_table;
ERROR: canceling statement due to user request
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_table;
count
-------
8
(1 row)
RESET SEARCH_PATH;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP SCHEMA insert_select_pushdown CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table insert_select_pushdown.events_table
drop cascades to table insert_select_pushdown.events_summary

View File

@ -0,0 +1,229 @@
--
-- failure_insert_select_via_coordinator
--
-- performs failure/cancellation test for insert/select executed by coordinator.
-- test for insert using CTEs are done in failure_cte_subquery, not repeating them here
--
CREATE SCHEMA coordinator_insert_select;
SET SEARCH_PATH=coordinator_insert_select;
SET citus.shard_count to 2;
SET citus.shard_replication_factor to 1;
SELECT pg_backend_pid() as pid \gset
CREATE TABLE events_table(user_id int, event_id int, event_type int);
CREATE TABLE events_summary(event_id int, event_type int, event_count int);
CREATE TABLE events_reference(event_type int, event_count int);
CREATE TABLE events_reference_distributed(event_type int, event_count int);
SELECT create_distributed_table('events_table', 'user_id');
create_distributed_table
--------------------------
(1 row)
SELECT create_distributed_table('events_summary', 'event_id');
create_distributed_table
--------------------------
(1 row)
SELECT create_reference_table('events_reference');
create_reference_table
------------------------
(1 row)
SELECT create_distributed_table('events_reference_distributed', 'event_type');
create_distributed_table
--------------------------
(1 row)
INSERT INTO events_table VALUES (1, 1, 3 ), (1, 2, 1), (1, 3, 2), (2, 4, 3), (3, 5, 1), (4, 7, 1), (4, 1, 9), (4, 3, 2);
SELECT count(*) FROM events_summary;
count
-------
0
(1 row)
-- insert/select from one distributed table to another
-- kill coordinator pull query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_summary SELECT event_id, event_type, count(*) FROM events_table GROUP BY 1,2;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- kill data push
SELECT citus.mitmproxy('conn.onQuery(query="^COPY coordinator_insert_select").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_summary SELECT event_id, event_type, count(*) FROM events_table GROUP BY 1,2;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- cancel coordinator pull query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_summary SELECT event_id, event_type, count(*) FROM events_table GROUP BY 1,2;
ERROR: canceling statement due to user request
-- cancel data push
SELECT citus.mitmproxy('conn.onQuery(query="^COPY coordinator_insert_select").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_summary SELECT event_id, event_type, count(*) FROM events_table GROUP BY 1,2;
ERROR: canceling statement due to user request
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_summary;
count
-------
0
(1 row)
-- insert into reference table from a distributed table
-- kill coordinator pull query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference SELECT event_type, count(*) FROM events_table GROUP BY 1;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- kill data push
SELECT citus.mitmproxy('conn.onQuery(query="^COPY coordinator_insert_select").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference SELECT event_type, count(*) FROM events_table GROUP BY 1;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- cancel coordinator pull query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference SELECT event_type, count(*) FROM events_table GROUP BY 1;
ERROR: canceling statement due to user request
-- cancel data push
SELECT citus.mitmproxy('conn.onQuery(query="^COPY coordinator_insert_select").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference SELECT event_type, count(*) FROM events_table GROUP BY 1;
ERROR: canceling statement due to user request
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_reference;
count
-------
0
(1 row)
-- insert/select from reference table to distributed
-- fill up reference table first
INSERT INTO events_reference SELECT event_type, count(*) FROM events_table GROUP BY 1;
-- kill coordinator pull query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference_distributed SELECT * FROM events_reference;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- kill data push
SELECT citus.mitmproxy('conn.onQuery(query="^COPY coordinator_insert_select").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference_distributed SELECT * FROM events_reference;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- cancel coordinator pull query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference_distributed SELECT * FROM events_reference;
ERROR: canceling statement due to user request
-- cancel data push
SELECT citus.mitmproxy('conn.onQuery(query="^COPY coordinator_insert_select").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO events_reference_distributed SELECT * FROM events_reference;
ERROR: canceling statement due to user request
--verify nothing is modified
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM events_reference_distributed;
count
-------
0
(1 row)
RESET SEARCH_PATH;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP SCHEMA coordinator_insert_select CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to table coordinator_insert_select.events_table
drop cascades to table coordinator_insert_select.events_summary
drop cascades to table coordinator_insert_select.events_reference
drop cascades to table coordinator_insert_select.events_reference_distributed

View File

@ -0,0 +1,490 @@
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_count = 2;
SET citus.shard_replication_factor = 1; -- one shard per worker
SET citus.next_shard_id TO 103400;
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 100;
CREATE TABLE dml_test (id integer, name text);
SELECT create_distributed_table('dml_test', 'id');
create_distributed_table
--------------------------
(1 row)
COPY dml_test FROM STDIN WITH CSV;
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
---- test multiple statements spanning multiple shards,
---- at each significant point. These transactions are 2pc
-- fail at DELETE
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
DELETE FROM dml_test WHERE id = 2;
ERROR: current transaction is aborted, commands ignored until end of transaction block
INSERT INTO dml_test VALUES (5, 'Epsilon');
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
--- shouldn't see any changes performed in failed transaction
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- cancel at DELETE
SELECT citus.mitmproxy('conn.onQuery(query="^DELETE").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
ERROR: canceling statement due to user request
DELETE FROM dml_test WHERE id = 2;
ERROR: current transaction is aborted, commands ignored until end of transaction block
INSERT INTO dml_test VALUES (5, 'Epsilon');
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
--- shouldn't see any changes performed in failed transaction
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- fail at INSERT
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
--- shouldn't see any changes before failed INSERT
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- cancel at INSERT
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
ERROR: canceling statement due to user request
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
--- shouldn't see any changes before failed INSERT
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- fail at UPDATE
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
--- shouldn't see any changes after failed UPDATE
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- cancel at UPDATE
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
ERROR: canceling statement due to user request
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
--- shouldn't see any changes after failed UPDATE
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- fail at PREPARE TRANSACTION
SELECT citus.mitmproxy('conn.onQuery(query="^PREPARE TRANSACTION").kill()');
mitmproxy
-----------
(1 row)
-- this transaction block will be sent to the coordinator as a remote command to hide the
-- error message that is caused during commit.
-- we'll test for the txn side-effects to ensure it didn't run
SELECT master_run_on_worker(
ARRAY['localhost']::text[],
ARRAY[:master_port]::int[],
ARRAY['
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, ''Epsilon'');
UPDATE dml_test SET name = ''alpha'' WHERE id = 1;
UPDATE dml_test SET name = ''gamma'' WHERE id = 3;
COMMIT;
'],
false
);
master_run_on_worker
---------------------------
(localhost,57636,t,BEGIN)
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
shardid
---------
(0 rows)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
0
(1 row)
-- shouldn't see any changes after failed PREPARE
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- cancel at PREPARE TRANSACTION
SELECT citus.mitmproxy('conn.onQuery(query="^PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
-- we'll test for the txn side-effects to ensure it didn't run
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
COMMIT;
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
shardid
---------
(0 rows)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
0
(1 row)
-- shouldn't see any changes after failed PREPARE
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- fail at COMMIT
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
mitmproxy
-----------
(1 row)
-- hide the error message (it has the PID)...
-- we'll test for the txn side-effects to ensure it didn't run
SET client_min_messages TO FATAL;
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
COMMIT;
SET client_min_messages TO DEFAULT;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
shardid
---------
(0 rows)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
1
(1 row)
-- should see changes, because of txn recovery
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+---------
3 | gamma
4 | Delta
5 | Epsilon
(3 rows)
-- cancel at COMMITs are ignored by Postgres
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
COMMIT;
-- should see changes, because cancellation is ignored
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+---------
3 | gamma
4 | Delta
5 | Epsilon
5 | Epsilon
(4 rows)
-- drop table and recreate with different replication/sharding
DROP TABLE dml_test;
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- two placements
CREATE TABLE dml_test (id integer, name text);
SELECT create_distributed_table('dml_test', 'id');
create_distributed_table
--------------------------
(1 row)
COPY dml_test FROM STDIN WITH CSV;
---- test multiple statements against a single shard, but with two placements
-- fail at COMMIT (actually COMMIT this time, as no 2pc in use)
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
COMMIT;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
--- should see all changes, but they only went to one placement (other is unhealthy)
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+---------
3 | gamma
4 | Delta
5 | Epsilon
(3 rows)
SELECT shardid FROM pg_dist_shard_placement WHERE shardstate = 3;
shardid
---------
103402
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- drop table and recreate as reference table
DROP TABLE dml_test;
SET citus.shard_count = 2;
SET citus.shard_replication_factor = 1;
CREATE TABLE dml_test (id integer, name text);
SELECT create_reference_table('dml_test');
create_reference_table
------------------------
(1 row)
COPY dml_test FROM STDIN WITH CSV;
-- fail at COMMIT (by failing to PREPARE)
SELECT citus.mitmproxy('conn.onQuery(query="^PREPARE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
COMMIT;
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
--- shouldn't see any changes after failed COMMIT
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- cancel at COMMIT (by cancelling on PREPARE)
SELECT citus.mitmproxy('conn.onQuery(query="^PREPARE").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM dml_test WHERE id = 1;
DELETE FROM dml_test WHERE id = 2;
INSERT INTO dml_test VALUES (5, 'Epsilon');
UPDATE dml_test SET name = 'alpha' WHERE id = 1;
UPDATE dml_test SET name = 'gamma' WHERE id = 3;
COMMIT;
ERROR: canceling statement due to user request
--- shouldn't see any changes after cancelled PREPARE
SELECT * FROM dml_test ORDER BY id ASC;
id | name
----+-------
1 | Alpha
2 | Beta
3 | Gamma
4 | Delta
(4 rows)
-- allow connection to allow DROP
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE dml_test;

View File

@ -0,0 +1,158 @@
--
-- failure_multi_row_insert
--
CREATE SCHEMA IF NOT EXISTS failure_multi_row_insert;
SET SEARCH_PATH TO failure_multi_row_insert;
-- this test is dependent on the shard count, so do not change
-- whitout changing the test
SET citus.shard_count TO 4;
SET citus.next_shard_id TO 301000;
SET citus.shard_replication_factor TO 1;
SELECT pg_backend_pid() as pid \gset
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE distributed_table(key int, value int);
CREATE TABLE reference_table(value int);
SELECT create_distributed_table('distributed_table', 'key');
create_distributed_table
--------------------------
(1 row)
SELECT create_reference_table('reference_table');
create_reference_table
------------------------
(1 row)
-- we'll test failure cases of the following cases:
-- (a) multi-row INSERT that hits the same shard with the same value
-- (b) multi-row INSERT that hits the same shard with different values
-- (c) multi-row INSERT that hits multiple shards in a single worker
-- (d) multi-row INSERT that hits multiple shards in multiple workers
-- (e) multi-row INSERT to a reference table
-- Failure and cancellation on multi-row INSERT that hits the same shard with the same value
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (1,1), (1,2), (1,3);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- this test is broken, see https://github.com/citusdata/citus/issues/2460
-- SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").cancel(' || :pid || ')');
-- INSERT INTO distributed_table VALUES (1,4), (1,5), (1,6);
-- Failure and cancellation on multi-row INSERT that hits the same shard with different values
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (1,7), (5,8);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- this test is broken, see https://github.com/citusdata/citus/issues/2460
-- SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").cancel(' || :pid || ')');
-- INSERT INTO distributed_table VALUES (1,9), (5,10);
-- Failure and cancellation multi-row INSERT that hits multiple shards in a single worker
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (1,11), (6,12);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (1,13), (6,14);
ERROR: canceling statement due to user request
-- Failure and cancellation multi-row INSERT that hits multiple shards in a single worker, happening on the second query
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").after(1).kill()');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (1,15), (6,16);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").after(1).cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (1,17), (6,18);
ERROR: canceling statement due to user request
-- Failure and cancellation multi-row INSERT that hits multiple shards in multiple workers
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (2,19),(1,20);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO distributed_table VALUES (2,21), (1,22);
ERROR: canceling statement due to user request
-- one test for the reference tables for completeness
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
INSERT INTO reference_table VALUES (1), (2), (3), (4);
ERROR: canceling statement due to user request
-- we've either failed or cancelled all queries, so should be empty
SELECT * FROM distributed_table;
key | value
-----+-------
(0 rows)
SELECT * FROM reference_table;
value
-------
(0 rows)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
RESET SEARCH_PATH;
DROP SCHEMA failure_multi_row_insert CASCADE;
NOTICE: drop cascades to 2 other objects
DETAIL: drop cascades to table failure_multi_row_insert.distributed_table
drop cascades to table failure_multi_row_insert.reference_table

View File

@ -6,6 +6,8 @@ SET SEARCH_PATH = multi_shard;
SET citus.shard_count TO 4;
SET citus.next_shard_id TO 201000;
SET citus.shard_replication_factor TO 1;
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 0;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------

View File

@ -0,0 +1,694 @@
--
-- failure_multi_shard_update_delete
--
CREATE SCHEMA IF NOT EXISTS multi_shard;
SET SEARCH_PATH = multi_shard;
SET citus.shard_count TO 4;
SET citus.next_shard_id TO 201000;
SET citus.shard_replication_factor TO 1;
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 0;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE t1(a int PRIMARY KEY, b int, c int);
CREATE TABLE r1(a int, b int PRIMARY KEY);
CREATE TABLE t2(a int REFERENCES t1(a) ON DELETE CASCADE, b int REFERENCES r1(b) ON DELETE CASCADE, c int);
SELECT create_distributed_table('t1', 'a');
create_distributed_table
--------------------------
(1 row)
SELECT create_reference_table('r1');
create_reference_table
------------------------
(1 row)
SELECT create_distributed_table('t2', 'a');
create_distributed_table
--------------------------
(1 row)
-- insert some data
INSERT INTO r1 VALUES (1, 1), (2, 2), (3, 3);
INSERT INTO t1 VALUES (1, 1, 1), (2, 2, 2), (3, 3, 3);
INSERT INTO t2 VALUES (1, 1, 1), (1, 2, 1), (2, 1, 2), (2, 2, 4), (3, 1, 3), (3, 2, 3), (3, 3, 3);
SELECT pg_backend_pid() as pid \gset
SELECT count(*) FROM t2;
count
-------
7
(1 row)
SHOW citus.multi_shard_commit_protocol ;
citus.multi_shard_commit_protocol
-----------------------------------
2pc
(1 row)
-- DELETION TESTS
-- delete using a filter on non-partition column filter
-- test both kill and cancellation
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM").kill()');
mitmproxy
-----------
(1 row)
-- issue a multi shard delete
DELETE FROM t2 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- kill just one connection
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM multi_shard.t2_201005").kill()');
mitmproxy
-----------
(1 row)
DELETE FROM t2 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- cancellation
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
-- issue a multi shard delete
DELETE FROM t2 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- cancel just one connection
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM multi_shard.t2_201005").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
DELETE FROM t2 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- UPDATE TESTS
-- update non-partition column based on a filter on another non-partition column
-- DELETION TESTS
-- delete using a filter on non-partition column filter
-- test both kill and cancellation
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
-- issue a multi shard update
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- kill just one connection
SELECT citus.mitmproxy('conn.onQuery(query="UPDATE multi_shard.t2_201005").kill()');
mitmproxy
-----------
(1 row)
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- cancellation
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
-- issue a multi shard update
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- cancel just one connection
SELECT citus.mitmproxy('conn.onQuery(query="UPDATE multi_shard.t2_201005").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- switch to 1PC
SET citus.multi_shard_commit_protocol TO '1PC';
-- DELETION TESTS
-- delete using a filter on non-partition column filter
-- test both kill and cancellation
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM").kill()');
mitmproxy
-----------
(1 row)
-- issue a multi shard delete
DELETE FROM t2 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- kill just one connection
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM multi_shard.t2_201005").kill()');
mitmproxy
-----------
(1 row)
DELETE FROM t2 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- cancellation
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
-- issue a multi shard delete
DELETE FROM t2 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- cancel just one connection
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM multi_shard.t2_201005").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
DELETE FROM t2 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is deleted
SELECT count(*) FROM t2;
count
-------
7
(1 row)
-- UPDATE TESTS
-- update non-partition column based on a filter on another non-partition column
-- DELETION TESTS
-- delete using a filter on non-partition column filter
-- test both kill and cancellation
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
-- issue a multi shard update
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- kill just one connection
SELECT citus.mitmproxy('conn.onQuery(query="UPDATE multi_shard.t2_201005").kill()');
mitmproxy
-----------
(1 row)
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- cancellation
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
-- issue a multi shard update
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
-- cancel just one connection
SELECT citus.mitmproxy('conn.onQuery(query="UPDATE multi_shard.t2_201005").cancel(' || :pid || ')');
mitmproxy
-----------
(1 row)
UPDATE t2 SET c = 4 WHERE b = 2;
ERROR: canceling statement due to user request
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 2) AS b2, count(*) FILTER (WHERE c = 4) AS c4 FROM t2;
b2 | c4
----+----
3 | 1
(1 row)
RESET citus.multi_shard_commit_protocol;
--
-- fail when cascading deletes from foreign key
-- unfortunately cascading deletes from foreign keys
-- are done inside the worker only and do not
-- generate any network output
-- therefore we can't just fail cascade part
-- following tests are added for completeness purposes
-- it is safe to remove them without reducing any
-- test coverage
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
-- check counts before delete
SELECT count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b2
----
3
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM").kill()');
mitmproxy
-----------
(1 row)
DELETE FROM r1 WHERE a = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is deleted
SELECT count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b2
----
3
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="DELETE FROM").kill()');
mitmproxy
-----------
(1 row)
DELETE FROM t2 WHERE b = 2;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is deleted
SELECT count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b2
----
3
(1 row)
-- test update with subquery pull
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE t3 AS SELECT * FROM t2;
SELECT create_distributed_table('t3', 'a');
NOTICE: Copying data from local table...
create_distributed_table
--------------------------
(1 row)
SELECT * FROM t3 ORDER BY 1, 2, 3;
a | b | c
---+---+---
1 | 1 | 1
1 | 2 | 1
2 | 1 | 2
2 | 2 | 4
3 | 1 | 3
3 | 2 | 3
3 | 3 | 3
(7 rows)
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
mitmproxy
-----------
(1 row)
UPDATE t3 SET c = q.c FROM (
SELECT b, max(c) as c FROM t2 GROUP BY b) q
WHERE t3.b = q.b
RETURNING *;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
--- verify nothing is updated
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT * FROM t3 ORDER BY 1, 2, 3;
a | b | c
---+---+---
1 | 1 | 1
1 | 2 | 1
2 | 1 | 2
2 | 2 | 4
3 | 1 | 3
3 | 2 | 3
3 | 3 | 3
(7 rows)
-- kill update part
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE multi_shard.t3_201009").kill()');
mitmproxy
-----------
(1 row)
UPDATE t3 SET c = q.c FROM (
SELECT b, max(c) as c FROM t2 GROUP BY b) q
WHERE t3.b = q.b
RETURNING *;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
--- verify nothing is updated
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT * FROM t3 ORDER BY 1, 2, 3;
a | b | c
---+---+---
1 | 1 | 1
1 | 2 | 1
2 | 1 | 2
2 | 2 | 4
3 | 1 | 3
3 | 2 | 3
3 | 3 | 3
(7 rows)
-- test with replication_factor = 2
-- table can not have foreign reference with this setting so
-- use a different set of table
SET citus.shard_replication_factor to 2;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE t3;
CREATE TABLE t3 AS SELECT * FROM t2;
SELECT create_distributed_table('t3', 'a');
NOTICE: Copying data from local table...
create_distributed_table
--------------------------
(1 row)
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
-- prevent update of one replica of one shard
SELECT citus.mitmproxy('conn.onQuery(query="UPDATE multi_shard.t3_201013").kill()');
mitmproxy
-----------
(1 row)
UPDATE t3 SET b = 2 WHERE b = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
-- fail only one update verify transaction is rolled back correctly
BEGIN;
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b1 | b2
----+----
3 | 3
(1 row)
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
UPDATE t2 SET b = 2 WHERE b = 1;
-- verify update is performed on t2
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b1 | b2
----+----
0 | 6
(1 row)
-- following will fail
UPDATE t3 SET b = 2 WHERE b = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
END;
-- verify everything is rolled back
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b1 | b2
----+----
3 | 3
(1 row)
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
UPDATE t3 SET b = 1 WHERE b = 2 RETURNING *;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
-- switch to 1PC
SET citus.multi_shard_commit_protocol TO '1PC';
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
UPDATE t3 SET b = 2 WHERE b = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- verify nothing is updated
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
-- fail only one update verify transaction is rolled back correctly
BEGIN;
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b1 | b2
----+----
3 | 3
(1 row)
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
UPDATE t2 SET b = 2 WHERE b = 1;
-- verify update is performed on t2
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b1 | b2
----+----
0 | 6
(1 row)
-- following will fail
UPDATE t3 SET b = 2 WHERE b = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
END;
-- verify everything is rolled back
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t2;
b1 | b2
----+----
3 | 3
(1 row)
SELECT count(*) FILTER (WHERE b = 1) b1, count(*) FILTER (WHERE b = 2) AS b2 FROM t3;
b1 | b2
----+----
3 | 3
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
RESET SEARCH_PATH;
DROP SCHEMA multi_shard CASCADE;
NOTICE: drop cascades to 4 other objects
DETAIL: drop cascades to table multi_shard.t1
drop cascades to table multi_shard.r1
drop cascades to table multi_shard.t2
drop cascades to table multi_shard.t3

View File

@ -26,18 +26,17 @@ INSERT INTO test_table VALUES(1,1,1),(1,2,2),(2,1,1),(2,2,2),(3,1,1),(3,2,2);
-- Kill when the first COPY command arrived, since we have a single placement
-- it is expected to error out.
SET client_min_messages TO ERROR;
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT public.raise_failed_execution('SELECT count(*) FROM test_table');
ERROR: Task failed to execute
CONTEXT: PL/pgSQL function public.raise_failed_execution(text) line 6 at RAISE
SELECT count(*) FROM test_table;
ERROR: failed to execute task 1
SET client_min_messages TO DEFAULT;
-- Kill the connection with a CTE
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").kill()');
mitmproxy
-----------
@ -47,29 +46,34 @@ WITH
results AS (SELECT * FROM test_table)
SELECT * FROM test_table, results
WHERE test_table.id = results.id;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
ERROR: failed to execute task 1
-- Since the outer query uses the connection opened by the CTE,
-- killing connection after first successful query should break.
SET client_min_messages TO ERROR;
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
(1 row)
SELECT public.raise_failed_execution('WITH
results AS (SELECT * FROM test_table)
SELECT * FROM test_table, results
WHERE test_table.id = results.id');
ERROR: Task failed to execute
CONTEXT: PL/pgSQL function public.raise_failed_execution(text) line 6 at RAISE
WITH results AS (SELECT * FROM test_table)
SELECT * FROM test_table, results
WHERE test_table.id = results.id;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SET client_min_messages TO DEFAULT;
-- In parallel execution mode Citus opens separate connections for each shard
-- so killing the connection after the first copy does not break it.
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).kill()');
SET citus.force_max_query_parallelization=ON;
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
@ -81,8 +85,10 @@ SELECT count(*) FROM test_table;
6
(1 row)
-- set back the force flag to original value
SET citus.force_max_query_parallelization=OFF;
-- Cancel a real-time executor query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -91,7 +97,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid()
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
-- Cancel a query within the transaction
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -102,7 +108,7 @@ SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query within the transaction after a multi-shard update
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -114,7 +120,7 @@ SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query with CTE
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -125,11 +131,9 @@ results AS (SELECT * FROM test_table)
SELECT * FROM test_table
WHERE test_table.id > (SELECT id FROM results);
ERROR: canceling statement due to user request
-- Since Citus opens a new connection after a failure within the real time
-- execution and after(1).kill() kills connection after a successful execution
-- for each connection, following transaciton does not fail.
-- Citus fails if the connection that is already used fails afterwards
SET citus.multi_shard_modify_mode to sequential;
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
@ -138,14 +142,14 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).kill()');
BEGIN;
SELECT count(*) FROM test_table;
WARNING: could not consume data from worker node
count
-------
6
(1 row)
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
ERROR: failed to execute task 1
COMMIT;
-- Cancel a real-time executor query - in sequential mode
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -154,7 +158,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid()
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
-- Cancel a query within the transaction - in sequential mode
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -165,7 +169,7 @@ SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel the query within a transaction after a single succesful run
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -180,6 +184,12 @@ COMMIT;
DROP TABLE test_table;
SET citus.multi_shard_modify_mode to default;
-- Create table with shard placements on each node
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_replication_factor to 2;
CREATE TABLE test_table(id int, value_1 int, value_2 int);
SELECT create_distributed_table('test_table','id');
@ -190,9 +200,9 @@ SELECT create_distributed_table('test_table','id');
-- Populate data to the table
INSERT INTO test_table VALUES(1,1,1),(1,2,2),(2,1,1),(2,2,2),(3,1,1),(3,2,2);
-- Kill when the first COPY command arrived, since we have placements on each node
-- Kill when the first SELECT command arrived, since we have placements on each node
-- it shouldn't fail.
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
@ -208,7 +218,7 @@ WARNING: could not consume data from worker node
-- Kill within the transaction, since we have placements on each node
-- it shouldn't fail.
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
@ -225,7 +235,7 @@ WARNING: could not consume data from worker node
COMMIT;
-- Cancel a real-time executor query
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -234,7 +244,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid()
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
-- Cancel a query within the transaction
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -245,7 +255,7 @@ SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query within the transaction after a multi-shard update
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -257,7 +267,7 @@ SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query with CTE
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").cancel(' || pg_backend_pid() || ')');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
@ -271,7 +281,7 @@ ERROR: canceling statement due to user request
-- Since we have the placement on each node, test with sequential mode
-- should pass as well.
SET citus.multi_shard_modify_mode to sequential;
SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).kill()');
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
@ -279,6 +289,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COPY").after(1).kill()');
BEGIN;
SELECT count(*) FROM test_table;
WARNING: could not consume data from worker node
WARNING: could not consume data from worker node
count
-------

View File

@ -0,0 +1,314 @@
--
-- Failure tests for real time select queries
--
CREATE SCHEMA real_time_select_failure;
SET search_path TO 'real_time_select_failure';
SET citus.next_shard_id TO 190000;
-- Preparation
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_count to 4;
SET citus.shard_replication_factor to 1;
-- create tables
CREATE TABLE test_table(id int, value_1 int, value_2 int);
SELECT create_distributed_table('test_table','id');
create_distributed_table
--------------------------
(1 row)
-- Populate data to the table
INSERT INTO test_table VALUES(1,1,1),(1,2,2),(2,1,1),(2,2,2),(3,1,1),(3,2,2);
-- Kill when the first COPY command arrived, since we have a single placement
-- it is expected to error out.
SET client_min_messages TO ERROR;
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM test_table;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SET client_min_messages TO DEFAULT;
-- Kill the connection with a CTE
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").kill()');
mitmproxy
-----------
(1 row)
WITH
results AS (SELECT * FROM test_table)
SELECT * FROM test_table, results
WHERE test_table.id = results.id;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- Since the outer query uses the connection opened by the CTE,
-- killing connection after first successful query should break.
SET client_min_messages TO ERROR;
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
(1 row)
WITH results AS (SELECT * FROM test_table)
SELECT * FROM test_table, results
WHERE test_table.id = results.id;
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
SET client_min_messages TO DEFAULT;
-- In parallel execution mode Citus opens separate connections for each shard
-- so killing the connection after the first copy does not break it.
SET citus.force_max_query_parallelization=ON;
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM test_table;
count
-------
6
(1 row)
-- set back the force flag to original value
SET citus.force_max_query_parallelization=OFF;
-- Cancel a real-time executor query
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
-- Cancel a query within the transaction
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query within the transaction after a multi-shard update
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
UPDATE test_table SET value_1 = value_1 + 1;
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query with CTE
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
WITH
results AS (SELECT * FROM test_table)
SELECT * FROM test_table
WHERE test_table.id > (SELECT id FROM results);
ERROR: canceling statement due to user request
-- Citus fails if the connection that is already used fails afterwards
SET citus.multi_shard_modify_mode to sequential;
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
COMMIT;
-- Cancel a real-time executor query - in sequential mode
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
-- Cancel a query within the transaction - in sequential mode
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel the query within a transaction after a single succesful run
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Now, test with replication factor 2, tests are expected to pass
-- since we have two placements for the same shard
DROP TABLE test_table;
SET citus.multi_shard_modify_mode to default;
-- Create table with shard placements on each node
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_replication_factor to 2;
CREATE TABLE test_table(id int, value_1 int, value_2 int);
SELECT create_distributed_table('test_table','id');
create_distributed_table
--------------------------
(1 row)
-- Populate data to the table
INSERT INTO test_table VALUES(1,1,1),(1,2,2),(2,1,1),(2,2,2),(3,1,1),(3,2,2);
-- Kill when the first SELECT command arrived, since we have placements on each node
-- it shouldn't fail.
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM test_table;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
count
-------
6
(1 row)
-- Kill within the transaction, since we have placements on each node
-- it shouldn't fail.
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
count
-------
6
(1 row)
COMMIT;
-- Cancel a real-time executor query
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
-- Cancel a query within the transaction
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query within the transaction after a multi-shard update
SELECT citus.mitmproxy('conn.onQuery(query=".*SELECT.*test_table.*").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
UPDATE test_table SET value_1 = value_1 + 1;
SELECT count(*) FROM test_table;
ERROR: canceling statement due to user request
COMMIT;
-- Cancel a query with CTE
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
WITH
results AS (SELECT * FROM test_table)
SELECT * FROM test_table
WHERE test_table.id > (SELECT id FROM results);
ERROR: canceling statement due to user request
-- Since we have the placement on each node, test with sequential mode
-- should pass as well.
SET citus.multi_shard_modify_mode to sequential;
SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SELECT count(*) FROM test_table;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
count
-------
6
(1 row)
COMMIT;
DROP SCHEMA real_time_select_failure CASCADE;
NOTICE: drop cascades to table test_table
SET search_path TO default;

View File

@ -0,0 +1,99 @@
SET citus.next_shard_id TO 100500;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE ref_table (key int, value int);
SELECT create_reference_table('ref_table');
create_reference_table
------------------------
(1 row)
\copy ref_table FROM stdin delimiter ',';
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
SELECT COUNT(*) FROM ref_table;
count
-------
4
(1 row)
-- verify behavior of single INSERT; should fail to execute
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO ref_table VALUES (5, 6);
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT COUNT(*) FROM ref_table WHERE key=5;
count
-------
0
(1 row)
-- verify behavior of UPDATE ... RETURNING; should not execute
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
UPDATE ref_table SET key=7 RETURNING value;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT COUNT(*) FROM ref_table WHERE key=7;
count
-------
0
(1 row)
-- verify fix to #2214; should raise error and fail to execute
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
DELETE FROM ref_table WHERE key=5;
UPDATE ref_table SET key=value;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
COMMIT;
SELECT COUNT(*) FROM ref_table WHERE key=value;
count
-------
0
(1 row)
-- all shards should still be healthy
SELECT COUNT(*) FROM pg_dist_shard_placement WHERE shardstate = 3;
count
-------
0
(1 row)
-- ==== Clean up, we're done here ====
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE ref_table;

View File

@ -1,3 +1,8 @@
-- We have two different output files for this failure test because the
-- failure behaviour of SAVEPOINT and RELEASE commands are different if
-- we use the executor. If we use it, these commands error out if any of
-- the placement commands fail. Otherwise, we might mark the placement
-- as invalid and continue with a WARNING.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------

View File

@ -0,0 +1,353 @@
-- We have two different output files for this failure test because the
-- failure behaviour of SAVEPOINT and RELEASE commands are different if
-- we use the executor. If we use it, these commands error out if any of
-- the placement commands fail. Otherwise, we might mark the placement
-- as invalid and continue with a WARNING.
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_count = 2;
SET citus.shard_replication_factor = 1; -- one shard per worker
SET citus.next_shard_id TO 100950;
ALTER SEQUENCE pg_catalog.pg_dist_placement_placementid_seq RESTART 150;
CREATE TABLE artists (
id bigint NOT NULL,
name text NOT NULL
);
SELECT create_distributed_table('artists', 'id');
create_distributed_table
--------------------------
(1 row)
-- add some data
INSERT INTO artists VALUES (1, 'Pablo Picasso');
INSERT INTO artists VALUES (2, 'Vincent van Gogh');
INSERT INTO artists VALUES (3, 'Claude Monet');
INSERT INTO artists VALUES (4, 'William Kurelek');
-- simply fail at SAVEPOINT
SELECT citus.mitmproxy('conn.onQuery(query="^SAVEPOINT").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO artists VALUES (5, 'Asher Lev');
SAVEPOINT s1;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection error: localhost:9060
DETAIL: connection not open
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
DELETE FROM artists WHERE id=4;
ERROR: current transaction is aborted, commands ignored until end of transaction block
RELEASE SAVEPOINT s1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
SELECT * FROM artists WHERE id IN (4, 5);
id | name
----+-----------------
4 | William Kurelek
(1 row)
-- fail at RELEASE
SELECT citus.mitmproxy('conn.onQuery(query="^RELEASE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
UPDATE artists SET name='a';
SAVEPOINT s1;
DELETE FROM artists WHERE id=4;
RELEASE SAVEPOINT s1;
WARNING: AbortSubTransaction while in COMMIT state
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection error: localhost:9060
DETAIL: connection not open
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: savepoint "savepoint_2" does not exist
CONTEXT: while executing command on localhost:57637
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
ROLLBACK;
SELECT * FROM artists WHERE id IN (4, 5);
id | name
----+-----------------
4 | William Kurelek
(1 row)
-- fail at ROLLBACK
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO artists VALUES (5, 'Asher Lev');
SAVEPOINT s1;
DELETE FROM artists WHERE id=4;
ROLLBACK TO SAVEPOINT s1;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
COMMIT;
ERROR: could not make changes to shard 100950 on any node
SELECT * FROM artists WHERE id IN (4, 5);
id | name
----+-----------------
4 | William Kurelek
(1 row)
-- fail at second RELEASE
SELECT citus.mitmproxy('conn.onQuery(query="^RELEASE").after(1).kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SAVEPOINT s1;
DELETE FROM artists WHERE id=4;
RELEASE SAVEPOINT s1;
SAVEPOINT s2;
INSERT INTO artists VALUES (5, 'Jacob Kahn');
RELEASE SAVEPOINT s2;
WARNING: AbortSubTransaction while in COMMIT state
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection error: localhost:9060
DETAIL: connection not open
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
ERROR: connection not open
CONTEXT: while executing command on localhost:9060
COMMIT;
SELECT * FROM artists WHERE id IN (4, 5);
id | name
----+-----------------
4 | William Kurelek
(1 row)
-- fail at second ROLLBACK
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").after(1).kill()');
mitmproxy
-----------
(1 row)
BEGIN;
SAVEPOINT s1;
UPDATE artists SET name='A' WHERE id=4;
ROLLBACK TO SAVEPOINT s1;
SAVEPOINT s2;
DELETE FROM artists WHERE id=5;
ROLLBACK TO SAVEPOINT s2;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
COMMIT;
ERROR: could not make changes to shard 100950 on any node
SELECT * FROM artists WHERE id IN (4, 5);
id | name
----+-----------------
4 | William Kurelek
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^RELEASE").after(1).kill()');
mitmproxy
-----------
(1 row)
-- Release after rollback
BEGIN;
SAVEPOINT s1;
ROLLBACK TO s1;
RELEASE SAVEPOINT s1;
SAVEPOINT s2;
INSERT INTO artists VALUES (6, 'John J. Audubon');
INSERT INTO artists VALUES (7, 'Emily Carr');
ROLLBACK TO s2;
RELEASE SAVEPOINT s2;
COMMIT;
SELECT * FROM artists WHERE id=7;
id | name
----+------
(0 rows)
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');
mitmproxy
-----------
(1 row)
-- Recover from errors
\set VERBOSITY terse
BEGIN;
SAVEPOINT s1;
SAVEPOINT s2;
INSERT INTO artists VALUES (6, 'John J. Audubon');
INSERT INTO artists VALUES (7, 'Emily Carr');
INSERT INTO artists VALUES (7, 'Emily Carr');
ROLLBACK TO SAVEPOINT s1;
WARNING: connection not open
WARNING: connection not open
WARNING: connection not open
WARNING: connection error: localhost:9060
WARNING: connection not open
WARNING: connection not open
COMMIT;
ERROR: could not make changes to shard 100950 on any node
SELECT * FROM artists WHERE id=6;
id | name
----+------
(0 rows)
-- replication factor > 1
CREATE TABLE researchers (
id bigint NOT NULL,
lab_id int NOT NULL,
name text NOT NULL
);
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- single shard, on both workers
SELECT create_distributed_table('researchers', 'lab_id', 'hash');
create_distributed_table
--------------------------
(1 row)
-- simply fail at SAVEPOINT
SELECT citus.mitmproxy('conn.onQuery(query="^SAVEPOINT").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO researchers VALUES (7, 4, 'Jan Plaza');
SAVEPOINT s1;
WARNING: connection not open
WARNING: connection error: localhost:9060
WARNING: connection not open
WARNING: connection not open
ERROR: connection not open
INSERT INTO researchers VALUES (8, 4, 'Alonzo Church');
ERROR: current transaction is aborted, commands ignored until end of transaction block
ROLLBACK TO s1;
ERROR: savepoint "s1" does not exist
RELEASE SAVEPOINT s1;
ERROR: current transaction is aborted, commands ignored until end of transaction block
COMMIT;
-- should see correct results from healthy placement and one bad placement
SELECT * FROM researchers WHERE lab_id = 4;
id | lab_id | name
----+--------+------
(0 rows)
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardstate = 3 AND shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'researchers'::regclass
) RETURNING placementid;
placementid
-------------
(0 rows)
TRUNCATE researchers;
-- fail at rollback
SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO researchers VALUES (7, 4, 'Jan Plaza');
SAVEPOINT s1;
INSERT INTO researchers VALUES (8, 4, 'Alonzo Church');
ROLLBACK TO s1;
WARNING: connection not open
WARNING: connection not open
RELEASE SAVEPOINT s1;
COMMIT;
ERROR: failure on connection marked as essential: localhost:9060
-- should see correct results from healthy placement and one bad placement
SELECT * FROM researchers WHERE lab_id = 4;
id | lab_id | name
----+--------+------
(0 rows)
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardstate = 3 AND shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'researchers'::regclass
) RETURNING placementid;
placementid
-------------
(0 rows)
TRUNCATE researchers;
-- fail at release
SELECT citus.mitmproxy('conn.onQuery(query="^RELEASE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO researchers VALUES (7, 4, 'Jan Plaza');
SAVEPOINT s1;
INSERT INTO researchers VALUES (8, 4, 'Alonzo Church');
ROLLBACK TO s1;
RELEASE SAVEPOINT s1;
WARNING: AbortSubTransaction while in COMMIT state
WARNING: connection not open
WARNING: connection error: localhost:9060
WARNING: connection not open
WARNING: connection not open
WARNING: savepoint "savepoint_3" does not exist
ERROR: connection not open
COMMIT;
-- should see correct results from healthy placement and one bad placement
SELECT * FROM researchers WHERE lab_id = 4;
id | lab_id | name
----+--------+------
(0 rows)
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardstate = 3 AND shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'researchers'::regclass
) RETURNING placementid;
placementid
-------------
(0 rows)
TRUNCATE researchers;
-- clean up
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE artists;
DROP TABLE researchers;

View File

@ -0,0 +1,128 @@
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
SET citus.shard_count = 2;
SET citus.shard_replication_factor = 2;
CREATE TABLE mod_test (key int, value text);
SELECT create_distributed_table('mod_test', 'key');
create_distributed_table
--------------------------
(1 row)
-- verify behavior of single INSERT; should mark shard as failed
SELECT citus.mitmproxy('conn.onQuery(query="^INSERT").kill()');
mitmproxy
-----------
(1 row)
INSERT INTO mod_test VALUES (2, 6);
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT COUNT(*) FROM mod_test WHERE key=2;
count
-------
1
(1 row)
-- some clean up
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
) AND shardstate = 3 RETURNING placementid;
placementid
-------------
137
(1 row)
TRUNCATE mod_test;
-- verify behavior of UPDATE ... RETURNING; should mark as failed
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
INSERT INTO mod_test VALUES (2, 6);
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
UPDATE mod_test SET value='ok' WHERE key=2 RETURNING key;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
key
-----
2
(1 row)
SELECT COUNT(*) FROM mod_test WHERE value='ok';
count
-------
1
(1 row)
-- some clean up
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
) AND shardstate = 3 RETURNING placementid;
placementid
-------------
137
(1 row)
TRUNCATE mod_test;
-- verify behavior of multi-statement modifications to a single shard
-- should succeed but mark a placement as failed
SELECT citus.mitmproxy('conn.onQuery(query="^UPDATE").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO mod_test VALUES (2, 6);
INSERT INTO mod_test VALUES (2, 7);
DELETE FROM mod_test WHERE key=2 AND value = '7';
UPDATE mod_test SET value='ok' WHERE key=2;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
COMMIT;
SELECT COUNT(*) FROM mod_test WHERE key=2;
count
-------
1
(1 row)
-- some clean up
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'mod_test'::regclass
) AND shardstate = 3 RETURNING placementid;
placementid
-------------
137
(1 row)
TRUNCATE mod_test;
-- ==== Clean up, we're done here ====
DROP TABLE mod_test;

View File

@ -0,0 +1,238 @@
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
SET citus.shard_count = 2;
SET citus.shard_replication_factor = 2;
CREATE TABLE select_test (key int, value text);
SELECT create_distributed_table('select_test', 'key');
create_distributed_table
--------------------------
(1 row)
-- put data in shard for which mitm node is first placement
INSERT INTO select_test VALUES (2, 'test data');
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
mitmproxy
-----------
(1 row)
SELECT * FROM select_test WHERE key = 2;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
key | value
-----+-----------
2 | test data
(1 row)
SELECT * FROM select_test WHERE key = 2;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
key | value
-----+-----------
2 | test data
(1 row)
-- kill after first SELECT; txn should work (though placement marked bad)
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").kill()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO select_test VALUES (2, 'more data');
SELECT * FROM select_test WHERE key = 2;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
key | value
-----+-----------
2 | test data
2 | more data
(2 rows)
INSERT INTO select_test VALUES (2, 'even more data');
SELECT * FROM select_test WHERE key = 2;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
key | value
-----+----------------
2 | test data
2 | more data
2 | even more data
(3 rows)
COMMIT;
-- some clean up
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'select_test'::regclass
);
TRUNCATE select_test;
-- now the same tests with query cancellation
-- put data in shard for which mitm node is first placement
INSERT INTO select_test VALUES (2, 'test data');
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT * FROM select_test WHERE key = 2;
ERROR: canceling statement due to user request
SELECT * FROM select_test WHERE key = 2;
ERROR: canceling statement due to user request
-- cancel after first SELECT; txn should fail and nothing should be marked as invalid
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO select_test VALUES (2, 'more data');
SELECT * FROM select_test WHERE key = 2;
ERROR: canceling statement due to user request
COMMIT;
-- show that all placements are OK
SELECT DISTINCT shardstate FROM pg_dist_shard_placement
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'select_test'::regclass
);
shardstate
------------
1
(1 row)
TRUNCATE select_test;
-- cancel the second query
-- error after second SELECT; txn should fail
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO select_test VALUES (2, 'more data');
SELECT * FROM select_test WHERE key = 2;
key | value
-----+-----------
2 | more data
(1 row)
INSERT INTO select_test VALUES (2, 'even more data');
SELECT * FROM select_test WHERE key = 2;
ERROR: canceling statement due to user request
COMMIT;
-- error after second SELECT; txn should work (though placement marked bad)
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).reset()');
mitmproxy
-----------
(1 row)
BEGIN;
INSERT INTO select_test VALUES (2, 'more data');
SELECT * FROM select_test WHERE key = 2;
key | value
-----+-----------
2 | more data
(1 row)
INSERT INTO select_test VALUES (2, 'even more data');
SELECT * FROM select_test WHERE key = 2;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
key | value
-----+----------------
2 | more data
2 | even more data
(2 rows)
COMMIT;
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(2).kill()');
mitmproxy
-----------
(1 row)
SELECT recover_prepared_transactions();
recover_prepared_transactions
-------------------------------
0
(1 row)
SELECT recover_prepared_transactions();
ERROR: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
CONTEXT: while executing command on localhost:9060
-- bug from https://github.com/citusdata/citus/issues/1926
SET citus.max_cached_conns_per_worker TO 0; -- purge cache
DROP TABLE select_test;
SET citus.shard_count = 2;
SET citus.shard_replication_factor = 1;
CREATE TABLE select_test (key int, value text);
SELECT create_distributed_table('select_test', 'key');
create_distributed_table
--------------------------
(1 row)
SET citus.max_cached_conns_per_worker TO 1; -- allow connection to be cached
INSERT INTO select_test VALUES (1, 'test data');
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).kill()');
mitmproxy
-----------
(1 row)
SELECT * FROM select_test WHERE key = 1;
key | value
-----+-----------
1 | test data
(1 row)
SELECT * FROM select_test WHERE key = 1;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
-- now the same test with query cancellation
SELECT citus.mitmproxy('conn.onQuery(query="^SELECT").after(1).cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
SELECT * FROM select_test WHERE key = 1;
key | value
-----+-----------
1 | test data
(1 row)
SELECT * FROM select_test WHERE key = 1;
ERROR: canceling statement due to user request
-- ==== Clean up, we're done here ====
DROP TABLE select_test;

View File

@ -6,6 +6,10 @@ SET search_path TO 'truncate_failure';
SET citus.next_shard_id TO 120000;
-- we don't want to see the prepared transaction numbers in the warnings
SET client_min_messages TO ERROR;
-- do not cache any connections
SET citus.max_cached_conns_per_worker TO 0;
-- use a predictable number of connections per task
SET citus.force_max_query_parallelization TO on;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
@ -278,12 +282,12 @@ SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT").kill()');
TRUNCATE test_table;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: could not commit transaction for shard 120002 on any active node

File diff suppressed because it is too large Load Diff

View File

@ -1,3 +1,6 @@
-- We have different output files for the executor. This is because
-- we don't mark transactions with ANALYZE as critical anymore, and
-- get WARNINGs instead of ERRORs.
-- print whether we're using version > 10 to make version-specific tests clear
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten;
@ -59,10 +62,18 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
ANALYZE vacuum_test;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
-- ANALYZE transactions being critical is an open question, see #2430
-- show that we marked as INVALID on COMMIT FAILURE
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
shardid | shardstate
---------+------------
102093 | 3
(1 row)
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass

View File

@ -1,3 +1,6 @@
-- We have different output files for the executor. The executor
-- with PG10 behaves like non-executor PG11, and with PG11 it
-- behaves like non-executor PG10.
-- print whether we're using version > 10 to make version-specific tests clear
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten;
@ -59,7 +62,7 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
ANALYZE vacuum_test;
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
WARNING: failed to commit critical transaction on localhost:9060, metadata is likely out of sync
WARNING: failed to commit transaction on localhost:9060
WARNING: connection not open
CONTEXT: while executing command on localhost:9060
-- ANALYZE transactions being critical is an open question, see #2430

View File

@ -0,0 +1,141 @@
-- We have different output files for the executor. This is because
-- we don't mark transactions with ANALYZE as critical anymore, and
-- get WARNINGs instead of ERRORs.
-- print whether we're using version > 10 to make version-specific tests clear
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten;
version_above_ten
-------------------
f
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- one shard per worker
SET citus.multi_shard_commit_protocol TO '1pc';
CREATE TABLE vacuum_test (key int, value int);
SELECT create_distributed_table('vacuum_test', 'key');
create_distributed_table
--------------------------
(1 row)
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").kill()');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^ANALYZE").kill()');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
-- ANALYZE transactions being critical is an open question, see #2430
-- show that we marked as INVALID on COMMIT FAILURE
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
shardid | shardstate
---------+------------
102093 | 3
(1 row)
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass
);
-- the same tests with cancel
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test;
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^ANALYZE").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
ERROR: canceling statement due to user request
-- cancel during COMMIT should be ignored
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE other_vacuum_test (key int, value int);
SELECT create_distributed_table('other_vacuum_test', 'key');
create_distributed_table
--------------------------
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM.*other").kill()');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test, other_vacuum_test;
ERROR: syntax error at or near ","
LINE 1: VACUUM vacuum_test, other_vacuum_test;
^
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM.*other").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test, other_vacuum_test;
ERROR: syntax error at or near ","
LINE 1: VACUUM vacuum_test, other_vacuum_test;
^
-- ==== Clean up, we're done here ====
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE vacuum_test, other_vacuum_test;

View File

@ -0,0 +1,140 @@
-- We have different output files for the executor. This is because
-- we don't mark transactions with ANALYZE as critical anymore, and
-- get WARNINGs instead of ERRORs.
-- print whether we're using version > 10 to make version-specific tests clear
SHOW server_version \gset
SELECT substring(:'server_version', '\d+')::int > 10 AS version_above_ten;
version_above_ten
-------------------
t
(1 row)
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
SET citus.shard_count = 1;
SET citus.shard_replication_factor = 2; -- one shard per worker
SET citus.multi_shard_commit_protocol TO '1pc';
CREATE TABLE vacuum_test (key int, value int);
SELECT create_distributed_table('vacuum_test', 'key');
create_distributed_table
--------------------------
(1 row)
SELECT citus.clear_network_traffic();
clear_network_traffic
-----------------------
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").kill()');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^ANALYZE").kill()');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
WARNING: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
-- ANALYZE transactions being critical is an open question, see #2430
-- show that we marked as INVALID on COMMIT FAILURE
SELECT shardid, shardstate FROM pg_dist_shard_placement where shardstate != 1 AND
shardid in ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass);
shardid | shardstate
---------+------------
102093 | 3
(1 row)
UPDATE pg_dist_shard_placement SET shardstate = 1
WHERE shardid IN (
SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass
);
-- the same tests with cancel
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test;
ERROR: canceling statement due to user request
SELECT citus.mitmproxy('conn.onQuery(query="^ANALYZE").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
ERROR: canceling statement due to user request
-- cancel during COMMIT should be ignored
SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
ANALYZE vacuum_test;
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
CREATE TABLE other_vacuum_test (key int, value int);
SELECT create_distributed_table('other_vacuum_test', 'key');
create_distributed_table
--------------------------
(1 row)
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM.*other").kill()');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test, other_vacuum_test;
ERROR: connection error: localhost:9060
DETAIL: server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
SELECT citus.mitmproxy('conn.onQuery(query="^VACUUM.*other").cancel(' || pg_backend_pid() || ')');
mitmproxy
-----------
(1 row)
VACUUM vacuum_test, other_vacuum_test;
ERROR: canceling statement due to user request
-- ==== Clean up, we're done here ====
SELECT citus.mitmproxy('conn.allow()');
mitmproxy
-----------
(1 row)
DROP TABLE vacuum_test, other_vacuum_test;

View File

@ -78,17 +78,30 @@ ORDER BY x;
(3 rows)
END;
CREATE FUNCTION raise_failed_execution_int_result(query text) RETURNS void AS $$
BEGIN
EXECUTE query;
EXCEPTION WHEN OTHERS THEN
IF SQLERRM LIKE '%does not exist%' THEN
RAISE 'Task failed to execute';
ELSIF SQLERRM LIKE '%could not receive query results%' THEN
RAISE 'Task failed to execute';
END IF;
END;
$$LANGUAGE plpgsql;
-- don't print the worker port
\set VERBOSITY terse
SET client_min_messages TO ERROR;
-- files should now be cleaned up
SELECT x, x2
FROM interesting_squares JOIN (SELECT * FROM read_intermediate_result('squares', 'binary') AS res (x text, x2 int)) squares ON (x = interested_in)
WHERE user_id = 'jon'
ORDER BY x;
WARNING: result "squares" does not exist
WARNING: result "squares" does not exist
ERROR: could not receive query results
SELECT raise_failed_execution_int_result($$
SELECT x, x2
FROM interesting_squares JOIN (SELECT * FROM read_intermediate_result('squares', 'binary') AS res (x text, x2 int)) squares ON (x = interested_in)
WHERE user_id = 'jon'
ORDER BY x;
$$);
ERROR: Task failed to execute
\set VERBOSITY DEFAULT
SET client_min_messages TO DEFAULT;
-- try to read the file as text, will fail because of binary encoding
BEGIN;
SELECT create_intermediate_result('squares', 'SELECT s, s*s FROM generate_series(1,5) s');
@ -244,8 +257,9 @@ SELECT * FROM squares ORDER BY x;
(5 rows)
DROP SCHEMA intermediate_results CASCADE;
NOTICE: drop cascades to 4 other objects
NOTICE: drop cascades to 5 other objects
DETAIL: drop cascades to table interesting_squares
drop cascades to function raise_failed_execution_int_result(text)
drop cascades to type square_type
drop cascades to table stored_squares
drop cascades to table squares

View File

@ -1,38 +1,34 @@
Parsed test spec with 3 sessions
starting permutation: s1-begin s2-begin s3-begin s1-alter-table s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-alter-table s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s2-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s3-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s1-alter-table:
ALTER TABLE test_table ADD COLUMN x INT;
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
@ -41,24 +37,22 @@ query query_hostname query_hostport master_query_host_namemaster_query_
ALTER TABLE test_table ADD COLUMN x INT;
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT worker_apply_shard_ddl_command (105958, 'public', '
SELECT worker_apply_shard_ddl_command (102081, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (105957, 'public', '
SELECT worker_apply_shard_ddl_command (102080, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (105956, 'public', '
SELECT worker_apply_shard_ddl_command (102079, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (105955, 'public', '
SELECT worker_apply_shard_ddl_command (102078, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle Client ClientRead postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle Client ClientRead postgres regression
step s2-rollback:
ROLLBACK;
@ -69,39 +63,35 @@ step s3-rollback:
ROLLBACK;
starting permutation: s1-begin s2-begin s3-begin s1-insert s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-insert s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s2-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s3-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s1-insert:
INSERT INTO test_table VALUES (100, 100);
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
@ -110,13 +100,11 @@ query query_hostname query_hostport master_query_host_namemaster_query_
INSERT INTO test_table VALUES (100, 100);
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle Client ClientRead postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle Client ClientRead postgres regression
INSERT INTO public.test_table_105961 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
INSERT INTO public.test_table_102084 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback:
ROLLBACK;
@ -127,28 +115,24 @@ step s3-rollback:
ROLLBACK;
starting permutation: s1-begin s2-begin s3-begin s1-select s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-select s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s2-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s3-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s1-select:
SELECT count(*) FROM test_table;
@ -156,13 +140,13 @@ count
0
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
@ -171,16 +155,14 @@ query query_hostname query_hostport master_query_host_namemaster_query_
SELECT count(*) FROM test_table;
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle Client ClientRead postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle Client ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_105966 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_105965 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_105964 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_105963 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102089 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102088 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102087 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
COPY (SELECT count(*) AS count FROM test_table_102086 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback:
ROLLBACK;
@ -191,28 +173,24 @@ step s3-rollback:
ROLLBACK;
starting permutation: s1-begin s2-begin s3-begin s1-select-router s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-select-router s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s2-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s3-begin:
BEGIN;
-- we don't want to see any entries related to 2PC recovery
SET citus.max_cached_conns_per_worker TO 0;
step s1-select-router:
SELECT count(*) FROM test_table WHERE column1 = 55;
@ -220,13 +198,13 @@ count
0
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
@ -235,13 +213,11 @@ query query_hostname query_hostport master_query_host_namemaster_query_
SELECT count(*) FROM test_table WHERE column1 = 55;
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle Client ClientRead postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle Client ClientRead postgres regression
SELECT count(*) AS count FROM public.test_table_105968 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle Client ClientRead postgres regression
SELECT count(*) AS count FROM public.test_table_102091 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle Client ClientRead postgres regression
step s2-rollback:
ROLLBACK;

View File

@ -1,9 +1,14 @@
Parsed test spec with 3 sessions
starting permutation: s1-begin s2-begin s3-begin s1-alter-table s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-alter-table s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
@ -17,39 +22,37 @@ step s1-alter-table:
ALTER TABLE test_table ADD COLUMN x INT;
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
ALTER TABLE test_table ADD COLUMN x INT;
coordinator_host57636 coordinator_host57636 idle in transaction postgres regression
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT worker_apply_shard_ddl_command (105425, 'public', '
SELECT worker_apply_shard_ddl_command (102081, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transaction postgres regression
SELECT worker_apply_shard_ddl_command (105424, 'public', '
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102080, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transaction postgres regression
SELECT worker_apply_shard_ddl_command (105423, 'public', '
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102079, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57638 coordinator_host57636 idle in transaction postgres regression
SELECT worker_apply_shard_ddl_command (105422, 'public', '
')localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT worker_apply_shard_ddl_command (102078, 'public', '
ALTER TABLE test_table ADD COLUMN x INT;
')localhost 57637 coordinator_host57636 idle in transaction postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle postgres regression
')localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback:
ROLLBACK;
@ -60,10 +63,15 @@ step s3-rollback:
ROLLBACK;
starting permutation: s1-begin s2-begin s3-begin s1-insert s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-insert s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
@ -77,28 +85,26 @@ step s1-insert:
INSERT INTO test_table VALUES (100, 100);
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
INSERT INTO test_table VALUES (100, 100);
coordinator_host57636 coordinator_host57636 idle in transaction postgres regression
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle postgres regression
INSERT INTO public.test_table_105428 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transaction postgres regression
INSERT INTO public.test_table_102084 (column1, column2) VALUES (100, 100)localhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback:
ROLLBACK;
@ -109,10 +115,15 @@ step s3-rollback:
ROLLBACK;
starting permutation: s1-begin s2-begin s3-begin s1-select s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-select s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
@ -129,31 +140,29 @@ count
0
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT count(*) FROM test_table;
coordinator_host57636 coordinator_host57636 idle in transaction postgres regression
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle postgres regression
COPY (SELECT count(*) AS count FROM test_table_105433 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transaction postgres regression
COPY (SELECT count(*) AS count FROM test_table_105432 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transaction postgres regression
COPY (SELECT count(*) AS count FROM test_table_105431 test_table WHERE true) TO STDOUTlocalhost 57638 coordinator_host57636 idle in transaction postgres regression
COPY (SELECT count(*) AS count FROM test_table_105430 test_table WHERE true) TO STDOUTlocalhost 57637 coordinator_host57636 idle in transaction postgres regression
SELECT count(*) AS count FROM test_table_102089 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT count(*) AS count FROM test_table_102088 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT count(*) AS count FROM test_table_102087 test_table WHERE truelocalhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
SELECT count(*) AS count FROM test_table_102086 test_table WHERE truelocalhost 57637 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback:
ROLLBACK;
@ -164,10 +173,15 @@ step s3-rollback:
ROLLBACK;
starting permutation: s1-begin s2-begin s3-begin s1-select-router s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
starting permutation: s1-cache-connections s1-begin s2-begin s3-begin s1-select-router s2-sleep s2-view-dist s3-view-worker s2-rollback s1-commit s3-rollback
create_distributed_table
step s1-cache-connections:
SET citus.max_cached_conns_per_worker TO 4;
SET citus.force_max_query_parallelization TO on;
UPDATE test_table SET column2 = 0;
step s1-begin:
BEGIN;
@ -184,29 +198,26 @@ count
0
step s2-sleep:
SELECT pg_sleep(0.2);
SELECT pg_sleep(0.5);
pg_sleep
step s2-view-dist:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_dist_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT count(*) FROM test_table WHERE column1 = 55;
coordinator_host57636 coordinator_host57636 idle in transaction postgres regression
coordinator_host57636 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s3-view-worker:
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity ORDER BY query DESC;
SELECT query, query_hostname, query_hostport, master_query_host_name, master_query_host_port, state, wait_event_type, wait_event, usename, datname FROM citus_worker_stat_activity WHERE query NOT ILIKE '%pg_prepared_xacts%' AND query NOT ILIKE '%COMMIT%' ORDER BY query DESC;
query query_hostname query_hostport master_query_host_namemaster_query_host_portstate wait_event_typewait_event usename datname
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57638 0 idle postgres regression
SELECT gid FROM pg_prepared_xacts WHERE gid LIKE 'citus\_0\_%'localhost 57637 0 idle postgres regression
SELECT count(*) AS count FROM public.test_table_105435 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 0 idle postgres regression
COMMIT localhost 57637 0 idle postgres regression
SELECT count(*) AS count FROM public.test_table_102091 test_table WHERE (column1 OPERATOR(pg_catalog.=) 55)localhost 57638 coordinator_host57636 idle in transactionClient ClientRead postgres regression
step s2-rollback:
ROLLBACK;

View File

@ -9,6 +9,7 @@ step s1-insert:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -62,6 +63,7 @@ step s1-insert:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -112,6 +114,7 @@ step s1-load-cache:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -162,6 +165,7 @@ step s1-load-cache:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -212,6 +216,7 @@ step s1-load-cache:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -262,6 +267,7 @@ step s1-insert:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -312,6 +318,7 @@ step s1-insert:
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -359,6 +366,7 @@ nodeport success result
starting permutation: s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-insert s2-commit s1-commit s2-print-content
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -406,6 +414,7 @@ nodeport success result
starting permutation: s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-copy s2-commit s1-commit s2-print-content
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;
@ -453,6 +462,7 @@ nodeport success result
starting permutation: s1-begin s1-select s2-set-placement-inactive s2-begin s2-repair-placement s1-ddl s2-commit s1-commit s2-print-index-count
step s1-begin:
BEGIN;
SET LOCAL citus.select_opens_transaction_block TO off;
step s1-select:
SELECT count(*) FROM test_copy_placement_vs_modification WHERE x = 5;

View File

@ -77,18 +77,18 @@ step s1-get-current-transaction-id:
row
(0,305)
(0,174)
step s2-get-first-worker-active-transactions:
SELECT * FROM run_command_on_workers('SELECT row(initiator_node_identifier, transaction_number)
FROM
get_current_transaction_id();
get_all_active_transactions();
')
WHERE nodeport = 57637;
;
nodename nodeport success result
localhost 57637 t (0,0)
localhost 57637 t (0,174)
step s1-commit:
COMMIT;

View File

@ -29,11 +29,11 @@ step detector-dump-wait-edges:
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
308 307 f
177 176 f
transactionnumberwaitingtransactionnumbers
307
308 307
176
177 176
step s1-abort:
ABORT;
@ -77,14 +77,14 @@ step detector-dump-wait-edges:
waiting_transaction_numblocking_transaction_numblocking_transaction_waiting
312 311 f
313 311 f
313 312 t
181 180 f
182 180 f
182 181 t
transactionnumberwaitingtransactionnumbers
311
312 311
313 311,312
180
181 180
182 180,181
step s1-abort:
ABORT;

View File

@ -11,7 +11,7 @@ step s2-begin:
BEGIN;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -26,7 +26,7 @@ step s1-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -129,7 +129,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -162,7 +162,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -194,7 +194,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -228,7 +228,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -241,8 +241,8 @@ user_id value_1 value_2 value_3
5 3 10 17
7 41 10 23
6 3 11 25
1 20 12 25
7 3 12 18
1 20 12 25
3 26 13 18
5 17 14 4
3 11 78 18
@ -315,7 +315,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -354,7 +354,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3
@ -392,7 +392,7 @@ step s2-commit:
COMMIT;
step s2-select:
SELECT * FROM users_test_table ORDER BY value_2;
SELECT * FROM users_test_table ORDER BY value_2, value_3;
user_id value_1 value_2 value_3

View File

@ -16,7 +16,7 @@ step s1-commit:
COMMIT;
step s2-insert: <... completed>
error in steps s1-commit s2-insert: ERROR: duplicate key value violates unique constraint "test_locking_a_key_102337"
error in steps s1-commit s2-insert: ERROR: duplicate key value violates unique constraint "test_locking_a_key_102353"
step s2-commit:
COMMIT;

View File

@ -1,5 +1,5 @@
SET citus.enable_repartition_joins to ON;
SET citus.max_intermediate_result_size TO 3;
SET citus.max_intermediate_result_size TO 2;
-- should fail because the copy size is ~4kB for each cte
WITH cte AS
(
@ -9,7 +9,7 @@ cte2 AS (
SELECT * FROM events_table
)
SELECT cte.user_id, cte.value_2 FROM cte,cte2 ORDER BY 1,2 LIMIT 10;
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 3 kB)
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 2 kB)
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
SET citus.max_intermediate_result_size TO 9;
@ -39,42 +39,33 @@ ERROR: the intermediate result size exceeds citus.max_intermediate_result_size
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
-- router queries should be able to get limitted too
SET citus.max_intermediate_result_size TO 3;
SET citus.max_intermediate_result_size TO 2;
-- this should pass, since we fetch small portions in each subplan
with cte as (select * from users_table where user_id=1),
cte2 as (select * from users_table where user_id=2),
cte3 as (select * from users_table where user_id=3),
cte4 as (select * from users_table where user_id=4),
cte5 as (select * from users_table where user_id=5)
SELECT * FROM (
(select * from cte)
SELECT sum(c) FROM (
(select count(*) as c from cte)
UNION
(select * from cte2)
(select count(*) as c from cte2)
UNION
(select * from cte3)
(select count(*) as c from cte3)
UNION
(select * from cte4)
(select count(*) as c from cte4)
UNION
(select * from cte5)
)a ORDER BY 1,2,3,4,5 LIMIT 10;
user_id | time | value_1 | value_2 | value_3 | value_4
---------+---------------------------------+---------+---------+---------+---------
1 | Wed Nov 22 22:51:43.132261 2017 | 4 | 0 | 3 |
1 | Thu Nov 23 03:32:50.803031 2017 | 3 | 2 | 1 |
1 | Thu Nov 23 09:26:42.145043 2017 | 1 | 3 | 3 |
1 | Thu Nov 23 11:11:24.40789 2017 | 3 | 4 | 0 |
1 | Thu Nov 23 11:44:57.515981 2017 | 4 | 3 | 4 |
1 | Thu Nov 23 17:23:03.441394 2017 | 5 | 4 | 3 |
1 | Thu Nov 23 17:30:34.635085 2017 | 3 | 4 | 4 |
2 | Wed Nov 22 18:19:49.944985 2017 | 3 | 5 | 1 |
2 | Thu Nov 23 00:19:14.138058 2017 | 3 | 4 | 0 |
2 | Thu Nov 23 01:04:26.198826 2017 | 4 | 3 | 4 |
(10 rows)
(select count(*) as c from cte5)
) as foo;
sum
-----
91
(1 row)
-- if we fetch the same amount of data at once, it should fail
WITH cte AS (SELECT * FROM users_table WHERE user_id IN (1,2,3,4,5))
SELECT * FROM cte ORDER BY 1,2,3,4,5 LIMIT 10;
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 3 kB)
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 2 kB)
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
SET citus.max_intermediate_result_size TO 0;
@ -131,6 +122,7 @@ ERROR: the intermediate result size exceeds citus.max_intermediate_result_size
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
-- this will fail in real_time_executor
SET citus.max_intermediate_result_size TO 2;
WITH cte AS (
WITH cte2 AS (
SELECT * FROM users_table WHERE user_id IN (1, 2)
@ -145,7 +137,7 @@ cte4 AS (
)
SELECT * FROM cte UNION ALL
SELECT * FROM cte4 ORDER BY 1,2,3,4,5 LIMIT 5;
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 3 kB)
ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 2 kB)
DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place.
HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable.
SET citus.max_intermediate_result_size TO 1;

View File

@ -18,7 +18,7 @@ SELECT l_shipmode FROM lineitem WHERE l_partkey = 67310 OR l_partkey = 155190;
MAIL
(2 rows)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SELECT count(*) FROM lineitem;
count
-------

View File

@ -410,7 +410,7 @@ LIMIT 10 OFFSET 20;
1453 | 5
(10 rows)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SET client_min_messages TO DEBUG1;
-- Ensure that we push down LIMIT and OFFSET properly
-- No Group-By -> Push Down

View File

@ -396,7 +396,7 @@ LIMIT 10 OFFSET 20;
1453 | 5
(10 rows)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SET client_min_messages TO DEBUG1;
-- Ensure that we push down LIMIT and OFFSET properly
-- No Group-By -> Push Down

View File

@ -11,7 +11,7 @@ SELECT substring(:'server_version', '\d+')::int > 9 AS version_above_nine;
(1 row)
\a\t
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SET citus.explain_distributed_queries TO on;
-- Function that parses explain output as JSON
CREATE FUNCTION explain_json(query text)
@ -1052,7 +1052,7 @@ Aggregate
-> Aggregate
-> Seq Scan on lineitem_290001 lineitem
Filter: (l_orderkey > 9030)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
EXPLAIN EXECUTE router_executor_query;
Custom Scan (Citus Router) (cost=0.00..0.00 rows=0 width=0)

View File

@ -387,12 +387,18 @@ RENAME TO dummy_assign_function;
SET citus.shard_replication_factor to 1;
-- create_distributed_table command should fail
CREATE TABLE t1(a int, b int);
SELECT create_distributed_table('t1', 'a');
WARNING: function assign_distributed_transaction_id(integer, integer, unknown) does not exist
HINT: No function matches the given name and argument types. You might need to add explicit type casts.
CONTEXT: while executing command on localhost:57637
ERROR: connection error: localhost:57637
DETAIL: another command is already in progress
SET client_min_messages TO ERROR;
DO $$
BEGIN
BEGIN
SELECT create_distributed_table('t1', 'a');
EXCEPTION WHEN OTHERS THEN
RAISE 'create distributed table failed';
END;
END;
$$;
ERROR: create distributed table failed
CONTEXT: PL/pgSQL function inline_code_block line 6 at RAISE
\c regression
\c - - - :worker_1_port
DROP DATABASE another;

View File

@ -10,7 +10,7 @@ SELECT create_distributed_table('tab', 'a');
INSERT INTO tab (a, b) VALUES (1, 1);
INSERT INTO tab (a, b) VALUES (1, 2);
\c - - - :follower_master_port
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SELECT * FROM tab;
a | b
---+---

View File

@ -182,48 +182,57 @@ DEBUG: Plan 14 query after replacing subqueries and CTEs: SELECT table1.id, tab
-- The following tests will fail as we do not support all joins on
-- all kinds of functions
-- In other words, we cannot recursively plan the functions and hence
-- the query fails on the workers
SET client_min_messages TO ERROR;
\set VERBOSITY terse
-- function joins in CTE results can create lateral joins that are not supported
SELECT public.raise_failed_execution($cmd$
WITH one_row AS (
SELECT * FROM table1 WHERE id=52
)
SELECT table1.id, table1.data
FROM one_row, table1, next_k_integers(one_row.id, 5) next_five_ids
WHERE table1.id = next_five_ids;
$cmd$);
-- we execute the query within a function to consolidate the error messages
-- between different executors
CREATE FUNCTION raise_failed_execution_func_join(query text) RETURNS void AS $$
BEGIN
EXECUTE query;
EXCEPTION WHEN OTHERS THEN
IF SQLERRM LIKE 'failed to execute task%' THEN
RAISE 'Task failed to execute';
ELSIF SQLERRM LIKE '%does not exist%' THEN
RAISE 'Task failed to execute';
END IF;
END;
$$LANGUAGE plpgsql;
SELECT raise_failed_execution_func_join($$
WITH one_row AS (
SELECT * FROM table1 WHERE id=52
)
SELECT table1.id, table1.data
FROM one_row, table1, next_k_integers(one_row.id, 5) next_five_ids
WHERE table1.id = next_five_ids;
$$);
ERROR: Task failed to execute
CONTEXT: PL/pgSQL function public.raise_failed_execution(text) line 6 at RAISE
-- a user-defined immutable function
CREATE OR REPLACE FUNCTION the_answer_to_life()
RETURNS INTEGER IMMUTABLE AS 'SELECT 42' LANGUAGE SQL;
SELECT public.raise_failed_execution($cmd$
SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer)
$cmd$);
SELECT raise_failed_execution_func_join($$
SELECT * FROM table1 JOIN the_answer_to_life() the_answer ON (id = the_answer);
$$);
ERROR: Task failed to execute
SELECT raise_failed_execution_func_join($$
SELECT *
FROM table1
JOIN next_k_integers(10,5) WITH ORDINALITY next_integers
ON (id = next_integers.result);
$$);
ERROR: Task failed to execute
CONTEXT: PL/pgSQL function public.raise_failed_execution(text) line 6 at RAISE
-- WITH ORDINALITY clause
SELECT public.raise_failed_execution($cmd$
SELECT *
FROM table1
JOIN next_k_integers(10,5) WITH ORDINALITY next_integers
ON (id = next_integers.result)
ORDER BY id ASC;
$cmd$);
SELECT raise_failed_execution_func_join($$
SELECT *
FROM table1
JOIN next_k_integers(10,5) WITH ORDINALITY next_integers
ON (id = next_integers.result)
ORDER BY id ASC;
$$);
ERROR: Task failed to execute
CONTEXT: PL/pgSQL function public.raise_failed_execution(text) line 6 at RAISE
RESET client_min_messages;
DROP SCHEMA functions_in_joins CASCADE;
NOTICE: drop cascades to 11 other objects
DETAIL: drop cascades to table table1
drop cascades to sequence numbers
drop cascades to function add(integer,integer)
drop cascades to function increment(integer)
drop cascades to function next_k_integers(integer,integer)
drop cascades to function get_set_of_records()
drop cascades to function dup(integer)
drop cascades to function the_minimum_id()
drop cascades to type min_and_max
drop cascades to function max_and_min()
drop cascades to function the_answer_to_life()
NOTICE: drop cascades to 12 other objects
SET search_path TO DEFAULT;

View File

@ -1741,14 +1741,14 @@ ALTER TABLE raw_events_second DROP COLUMN value_4;
INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 100;
ROLLBACK;
-- Altering a reference table and then performing an INSERT ... SELECT which
-- joins with the reference table is not allowed, since the INSERT ... SELECT
-- would read from the reference table over others connections than the ones
-- joins with the reference table is allowed, since the INSERT ... SELECT
-- would read from the reference table over the same connections with the ones
-- that performed the parallel DDL.
BEGIN;
ALTER TABLE reference_table ADD COLUMN z int;
INSERT INTO raw_events_first (user_id)
SELECT user_id FROM raw_events_second JOIN reference_table USING (user_id);
ERROR: cannot establish a new connection for placement 13300024, since DDL has been executed on a connection that is in use
ERROR: cannot establish a new connection for placement 13300025, since DDL has been executed on a connection that is in use
ROLLBACK;
-- the same test with sequential DDL should work fine
BEGIN;

File diff suppressed because it is too large Load Diff

View File

@ -102,7 +102,7 @@ INSERT INTO append_partitioned VALUES (414123, 'AAPL', 9580, '2004-10-19 10:23:5
20.69);
-- ensure the values are where we put them and query to ensure they are properly pruned
SET client_min_messages TO 'DEBUG2';
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SELECT * FROM range_partitioned WHERE id = 32743;
DEBUG: Creating router plan
DEBUG: Plan is router executable
@ -347,10 +347,17 @@ ALTER TABLE limit_orders_750000 RENAME TO renamed_orders;
-- Third: Connect back to master node
\c - - - :master_port
-- Fourth: Perform an INSERT on the remaining node
-- the whole transaction should fail
\set VERBOSITY terse
INSERT INTO limit_orders VALUES (276, 'ADR', 140, '2007-07-02 16:32:15', 'sell', 43.67);
WARNING: relation "public.limit_orders_750000" does not exist
-- Last: Verify the insert worked but the deleted placement is now unhealthy
-- set the shard name back
\c - - - :worker_2_port
-- Second: Move aside limit_orders shard on the second worker node
ALTER TABLE renamed_orders RENAME TO limit_orders_750000;
-- Connect back to master node
\c - - - :master_port
-- Verify the insert failed and both placements are healthy
SELECT count(*) FROM limit_orders WHERE id = 276;
count
-------
@ -361,8 +368,6 @@ SELECT count(*)
FROM pg_dist_shard_placement AS sp,
pg_dist_shard AS s
WHERE sp.shardid = s.shardid
AND sp.nodename = 'localhost'
AND sp.nodeport = :worker_2_port
AND sp.shardstate = 3
AND s.logicalrelid = 'limit_orders'::regclass;
count

File diff suppressed because it is too large Load Diff

View File

@ -420,7 +420,7 @@ BEGIN;
\copy labs from stdin delimiter ','
ALTER TABLE labs ADD COLUMN motto text;
ABORT;
-- cannot perform parallel DDL once a connection is used for multiple shards
-- can perform parallel DDL even a connection is used for multiple shards
BEGIN;
SELECT lab_id FROM researchers WHERE lab_id = 1 AND id = 0;
lab_id
@ -523,14 +523,14 @@ DEFERRABLE INITIALLY IMMEDIATE
FOR EACH ROW EXECUTE PROCEDURE reject_bad();
\c - - - :master_port
-- test partial failure; worker_1 succeeds, 2 fails
-- in this case, we expect the transaction to abort
\set VERBOSITY terse
BEGIN;
INSERT INTO objects VALUES (1, 'apple');
INSERT INTO objects VALUES (2, 'BAD');
WARNING: illegal value
INSERT INTO labs VALUES (7, 'E Corp');
COMMIT;
-- data should be persisted
-- so the data should noy be persisted
SELECT * FROM objects WHERE id = 2;
id | name
----+------
@ -538,12 +538,11 @@ SELECT * FROM objects WHERE id = 2;
(1 row)
SELECT * FROM labs WHERE id = 7;
id | name
----+--------
7 | E Corp
(1 row)
id | name
----+------
(0 rows)
-- but one placement should be bad
-- and none of placements should be inactive
SELECT count(*)
FROM pg_dist_shard_placement AS sp,
pg_dist_shard AS s
@ -558,12 +557,8 @@ AND s.logicalrelid = 'objects'::regclass;
(1 row)
DELETE FROM objects;
-- mark shards as healthy again; delete all data
UPDATE pg_dist_shard_placement AS sp SET shardstate = 1
FROM pg_dist_shard AS s
WHERE sp.shardid = s.shardid
AND s.logicalrelid = 'objects'::regclass;
-- what if there are errors on different shards at different times?
-- there cannot be errors on different shards at different times
-- because the first failure will fail the whole transaction
\c - - - :worker_1_port
CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$
BEGIN
@ -582,9 +577,8 @@ FOR EACH ROW EXECUTE PROCEDURE reject_bad();
BEGIN;
INSERT INTO objects VALUES (1, 'apple');
INSERT INTO objects VALUES (2, 'BAD');
WARNING: illegal value
INSERT INTO labs VALUES (8, 'Aperture Science');
INSERT INTO labs VALUES (9, 'BAD');
INSERT INTO labs VALUES (2, 'BAD');
ERROR: illegal value
COMMIT;
-- data should NOT be persisted
@ -608,7 +602,7 @@ AND (s.logicalrelid = 'objects'::regclass OR
s.logicalrelid = 'labs'::regclass);
count
-------
3
2
(1 row)
-- what if the failures happen at COMMIT time?
@ -625,8 +619,6 @@ INSERT INTO objects VALUES (1, 'apple');
INSERT INTO objects VALUES (2, 'BAD');
INSERT INTO labs VALUES (9, 'Umbrella Corporation');
COMMIT;
WARNING: illegal value
WARNING: failed to commit transaction on localhost:57638
-- data should be persisted
SELECT * FROM objects WHERE id = 2;
id | name
@ -634,12 +626,6 @@ SELECT * FROM objects WHERE id = 2;
2 | BAD
(1 row)
SELECT * FROM labs WHERE id = 7;
id | name
----+--------
7 | E Corp
(1 row)
-- but one placement should be bad
SELECT count(*)
FROM pg_dist_shard_placement AS sp,
@ -1433,7 +1419,7 @@ INSERT INTO users VALUES (2, 'burak');
INSERT INTO users VALUES (3, 'burak');
\COPY items FROM STDIN WITH CSV
ROLLBACK;
-- cannot perform parallel DDL after a co-located table has been read over 1 connection
-- perform parallel DDL after a co-located table has been read over 1 connection
BEGIN;
SELECT id FROM users WHERE id = 1;
id
@ -1449,7 +1435,7 @@ SELECT id FROM users WHERE id = 6;
ALTER TABLE items ADD COLUMN last_update timestamptz;
ERROR: cannot perform a parallel DDL command because multiple placements have been accessed over the same connection
END;
ROLLBACK;
-- can perform sequential DDL after a co-located table has been read over 1 connection
BEGIN;
SET LOCAL citus.multi_shard_modify_mode TO 'sequential';
@ -1467,7 +1453,7 @@ SELECT id FROM users WHERE id = 6;
ALTER TABLE items ADD COLUMN last_update timestamptz;
ROLLBACK;
-- but the other way around is fine
-- and the other way around is also fine
BEGIN;
ALTER TABLE items ADD COLUMN last_update timestamptz;
SELECT id FROM users JOIN items ON (id = user_id) WHERE id = 1;
@ -1497,7 +1483,9 @@ SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id =
---------
(0 rows)
-- perform a DDL command on the reference table
-- perform a DDL command on the reference table errors
-- because the current implementation of COPY always opens one connection
-- per placement SELECTs have to use those connections for correctness
ALTER TABLE itemgroups ADD COLUMN last_update timestamptz;
ERROR: cannot perform DDL on placement 1200036, which has been read over multiple connections
END;

File diff suppressed because it is too large Load Diff

View File

@ -166,7 +166,7 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.
COPY "postgresql.conf" TO STDOUT WITH (format transmit);
ERROR: operation is not allowed
HINT: Run the command with a superuser.
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- should not be able to transmit directly
COPY "postgresql.conf" TO STDOUT WITH (format transmit);
ERROR: operation is not allowed
@ -239,7 +239,7 @@ SELECT lock_relation_if_exists('test', 'ACCESS SHARE');
SELECT lock_relation_if_exists('test', 'EXCLUSIVE');
ERROR: permission denied for table test
ABORT;
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- check no permission
SET ROLE no_access;
EXECUTE prepare_insert(1);
@ -262,7 +262,7 @@ ERROR: permission denied for table test
COPY "postgresql.conf" TO STDOUT WITH (format transmit);
ERROR: operation is not allowed
HINT: Run the command with a superuser.
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- should be able to use intermediate results as any user
BEGIN;
SELECT create_intermediate_result('topten', 'SELECT s FROM generate_series(1,10) s');

View File

@ -166,7 +166,7 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.
COPY "postgresql.conf" TO STDOUT WITH (format transmit);
ERROR: operation is not allowed
HINT: Run the command with a superuser.
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- should not be able to transmit directly
COPY "postgresql.conf" TO STDOUT WITH (format transmit);
ERROR: operation is not allowed
@ -239,7 +239,7 @@ SELECT lock_relation_if_exists('test', 'ACCESS SHARE');
SELECT lock_relation_if_exists('test', 'EXCLUSIVE');
ERROR: permission denied for relation test
ABORT;
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- check no permission
SET ROLE no_access;
EXECUTE prepare_insert(1);
@ -262,7 +262,7 @@ ERROR: permission denied for relation test
COPY "postgresql.conf" TO STDOUT WITH (format transmit);
ERROR: operation is not allowed
HINT: Run the command with a superuser.
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- should be able to use intermediate results as any user
BEGIN;
SELECT create_intermediate_result('topten', 'SELECT s FROM generate_series(1,10) s');

View File

@ -6,7 +6,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1320000;
\c - - - :worker_2_port
\c - - - :master_port
\a\t
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SET citus.explain_distributed_queries TO on;
VACUUM ANALYZE lineitem_mx;
VACUUM ANALYZE orders_mx;
@ -66,7 +66,7 @@ Sort
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), remote_scan.l_quantity
-> HashAggregate
Group Key: remote_scan.l_quantity
-> Custom Scan (Citus Real-Time)
-> Custom Scan (Citus Adaptive)
Task Count: 16
Tasks Shown: One of 16
-> Task
@ -96,7 +96,7 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
{
"Node Type": "Custom Scan",
"Parent Relationship": "Outer",
"Custom Plan Provider": "Citus Real-Time",
"Custom Plan Provider": "Citus Adaptive",
"Parallel Aware": false,
"Distributed Query": {
"Job": {
@ -172,7 +172,7 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
<Plan>
<Node-Type>Custom Scan</Node-Type>
<Parent-Relationship>Outer</Parent-Relationship>
<Custom-Plan-Provider>Citus Real-Time</Custom-Plan-Provider>
<Custom-Plan-Provider>Citus Adaptive</Custom-Plan-Provider>
<Parallel-Aware>false</Parallel-Aware>
<Distributed-Query>
<Job>
@ -242,7 +242,7 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Plans:
- Node Type: "Custom Scan"
Parent Relationship: "Outer"
Custom Plan Provider: "Citus Real-Time"
Custom Plan Provider: "Citus Adaptive"
Parallel Aware: false
Distributed Query:
Job:
@ -273,7 +273,7 @@ Sort
Sort Key: COALESCE((pg_catalog.sum((COALESCE((pg_catalog.sum(remote_scan.count_quantity))::bigint, '0'::bigint))))::bigint, '0'::bigint), remote_scan.l_quantity
-> HashAggregate
Group Key: remote_scan.l_quantity
-> Custom Scan (Citus Real-Time)
-> Custom Scan (Citus Adaptive)
Task Count: 16
Tasks Shown: One of 16
-> Task
@ -287,7 +287,7 @@ EXPLAIN (COSTS FALSE, VERBOSE TRUE)
SELECT sum(l_quantity) / avg(l_quantity) FROM lineitem_mx;
Aggregate
Output: (sum(remote_scan."?column?") / (sum(remote_scan."?column?_1") / pg_catalog.sum(remote_scan."?column?_2")))
-> Custom Scan (Citus Real-Time)
-> Custom Scan (Citus Adaptive)
Output: remote_scan."?column?", remote_scan."?column?_1", remote_scan."?column?_2"
Task Count: 16
Tasks Shown: One of 16
@ -305,7 +305,7 @@ EXPLAIN (COSTS FALSE)
Limit
-> Sort
Sort Key: remote_scan.l_quantity
-> Custom Scan (Citus Real-Time)
-> Custom Scan (Citus Adaptive)
Task Count: 16
Tasks Shown: One of 16
-> Task
@ -322,7 +322,7 @@ Limit
-- Test insert
EXPLAIN (COSTS FALSE)
INSERT INTO lineitem_mx VALUES(1,0);
Custom Scan (Citus Router)
Custom Scan (Citus Adaptive)
Task Count: 1
Tasks Shown: All
-> Task
@ -334,7 +334,7 @@ EXPLAIN (COSTS FALSE)
UPDATE lineitem_mx
SET l_suppkey = 12
WHERE l_orderkey = 1 AND l_partkey = 0;
Custom Scan (Citus Router)
Custom Scan (Citus Adaptive)
Task Count: 1
Tasks Shown: All
-> Task
@ -347,7 +347,7 @@ Custom Scan (Citus Router)
EXPLAIN (COSTS FALSE)
DELETE FROM lineitem_mx
WHERE l_orderkey = 1 AND l_partkey = 0;
Custom Scan (Citus Router)
Custom Scan (Citus Adaptive)
Task Count: 1
Tasks Shown: All
-> Task
@ -364,7 +364,7 @@ VACUUM ANALYZE supplier_mx;
-- Test single-shard SELECT
EXPLAIN (COSTS FALSE)
SELECT l_quantity FROM lineitem_mx WHERE l_orderkey = 5;
Custom Scan (Citus Router)
Custom Scan (Citus Adaptive)
Task Count: 1
Tasks Shown: All
-> Task
@ -381,7 +381,7 @@ t
EXPLAIN (COSTS FALSE)
CREATE TABLE explain_result AS
SELECT * FROM lineitem_mx;
Custom Scan (Citus Real-Time)
Custom Scan (Citus Adaptive)
Task Count: 16
Tasks Shown: One of 16
-> Task
@ -392,7 +392,7 @@ SET citus.explain_all_tasks TO on;
EXPLAIN (COSTS FALSE)
SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030;
Aggregate
-> Custom Scan (Citus Real-Time)
-> Custom Scan (Citus Adaptive)
Task Count: 16
Tasks Shown: All
-> Task

View File

@ -75,7 +75,7 @@ ORDER BY
(3 rows)
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
inhrelid
------------------------
partitioning_test_2009
@ -117,7 +117,7 @@ ORDER BY
(2 rows)
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
inhrelid
------------------------
partitioning_test_2009
@ -177,7 +177,7 @@ SELECT * FROM partitioning_test ORDER BY 1;
(6 rows)
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
inhrelid
------------------------
partitioning_test_2009
@ -217,7 +217,7 @@ SELECT * FROM partitioning_test ORDER BY 1;
(8 rows)
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
inhrelid
------------------------
partitioning_test_2009
@ -232,7 +232,7 @@ SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'
ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009;
-- see from MX node, partitioning hierarchy is built
\c - - - :worker_1_port
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
inhrelid
------------------------
partitioning_test_2010

View File

@ -79,7 +79,7 @@ ERROR: relation "partitioning_test" does not exist
LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t...
^
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
ERROR: relation "partitioning_test" does not exist
LINE 1: ...elid::regclass FROM pg_inherits WHERE inhparent = 'partition...
^
@ -115,7 +115,7 @@ ERROR: relation "partitioning_test" does not exist
LINE 4: WHERE logicalrelid IN ('partitioning_test', 'partitioning_t...
^
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
ERROR: relation "partitioning_test" does not exist
LINE 1: ...elid::regclass FROM pg_inherits WHERE inhparent = 'partition...
^
@ -160,7 +160,7 @@ ERROR: relation "partitioning_test" does not exist
LINE 1: SELECT * FROM partitioning_test ORDER BY 1;
^
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
ERROR: relation "partitioning_test" does not exist
LINE 1: ...elid::regclass FROM pg_inherits WHERE inhparent = 'partition...
^
@ -189,7 +189,7 @@ ERROR: relation "partitioning_test" does not exist
LINE 1: SELECT * FROM partitioning_test ORDER BY 1;
^
-- see from MX node, partitioning hierarchy is built
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
ERROR: relation "partitioning_test" does not exist
LINE 1: ...elid::regclass FROM pg_inherits WHERE inhparent = 'partition...
^
@ -201,7 +201,7 @@ LINE 1: ALTER TABLE partitioning_test DETACH PARTITION partitioning_...
^
-- see from MX node, partitioning hierarchy is built
\c - - - :worker_1_port
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass;
SELECT inhrelid::regclass FROM pg_inherits WHERE inhparent = 'partitioning_test'::regclass ORDER BY 1;
ERROR: relation "partitioning_test" does not exist
LINE 1: ...elid::regclass FROM pg_inherits WHERE inhparent = 'partition...
^

View File

@ -56,7 +56,7 @@ INSERT INTO articles_hash_mx VALUES (47, 7, 'abeyance', 1772);
INSERT INTO articles_hash_mx VALUES (48, 8, 'alkylic', 18610);
INSERT INTO articles_hash_mx VALUES (49, 9, 'anyone', 2681);
INSERT INTO articles_hash_mx VALUES (50, 10, 'anjanette', 19519);
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
SET client_min_messages TO 'DEBUG2';
-- insert a single row for the test
INSERT INTO articles_single_shard_hash_mx VALUES (50, 10, 'anjanette', 19519);
@ -879,7 +879,7 @@ ORDER BY 1,2,3,4;
43 | 3 | affixal | 12723
(15 rows)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- Test various filtering options for router plannable check
SET client_min_messages to 'DEBUG2';
-- this is definitely single shard

View File

@ -16,7 +16,7 @@ SET client_min_messages TO DEBUG2;
SET citus.explain_all_tasks TO on;
-- to avoid differing explain output - executor doesn't matter,
-- because were testing pruning here.
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- Change configuration to treat lineitem and orders tables as large
SET citus.log_multi_join_order to true;
SET citus.enable_repartition_joins to ON;

View File

@ -16,7 +16,7 @@ SET client_min_messages TO DEBUG2;
SET citus.explain_all_tasks TO on;
-- to avoid differing explain output - executor doesn't matter,
-- because were testing pruning here.
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- Change configuration to treat lineitem and orders tables as large
SET citus.log_multi_join_order to true;
SET citus.enable_repartition_joins to ON;
@ -40,7 +40,7 @@ DEBUG: Router planner does not support append-partitioned tables.
LOG: join order: [ "lineitem" ]
QUERY PLAN
-----------------------------------------------------------------------
Custom Scan (Citus Real-Time)
Custom Scan (Citus Adaptive)
Task Count: 2
Tasks Shown: All
-> Task
@ -73,7 +73,7 @@ DEBUG: join prunable for intervals [8997,14947] and [1,5986]
QUERY PLAN
--------------------------------------------------------------------------------
Aggregate
-> Custom Scan (Citus Real-Time)
-> Custom Scan (Citus Adaptive)
Task Count: 2
Tasks Shown: All
-> Task
@ -104,7 +104,7 @@ DEBUG: Router planner does not support append-partitioned tables.
LOG: join order: [ "lineitem" ]
QUERY PLAN
-------------------------------------------------------------------------------
Custom Scan (Citus Real-Time)
Custom Scan (Citus Adaptive)
Task Count: 2
Tasks Shown: All
-> Task
@ -176,7 +176,7 @@ DEBUG: Router planner does not support append-partitioned tables.
LOG: join order: [ "lineitem" ]
QUERY PLAN
-------------------------------------------------------------------------------
Custom Scan (Citus Real-Time)
Custom Scan (Citus Adaptive)
Task Count: 2
Tasks Shown: All
-> Task
@ -249,7 +249,7 @@ LOG: join order: [ "lineitem" ]
DEBUG: Plan is router executable
QUERY PLAN
-------------------------------------------------------------------------------
Custom Scan (Citus Router)
Custom Scan (Citus Adaptive)
Task Count: 1
Tasks Shown: All
-> Task

View File

@ -1302,7 +1302,7 @@ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass
(3 rows)
COMMIT;
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- test locks on INSERT
BEGIN;
INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01');

View File

@ -1274,7 +1274,7 @@ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass
(3 rows)
COMMIT;
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- test locks on INSERT
BEGIN;
INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01');
@ -1394,12 +1394,10 @@ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass
partitioning_locks | relation | AccessExclusiveLock
partitioning_locks | relation | AccessShareLock
partitioning_locks_2009 | relation | AccessExclusiveLock
partitioning_locks_2009 | relation | AccessShareLock
partitioning_locks_2009 | relation | ShareLock
partitioning_locks_2010 | relation | AccessExclusiveLock
partitioning_locks_2010 | relation | AccessShareLock
partitioning_locks_2010 | relation | ShareLock
(8 rows)
(6 rows)
COMMIT;
-- test shard resource locks with multi-shard UPDATE
@ -1453,21 +1451,13 @@ WHERE
pid = pg_backend_pid()
ORDER BY
1, 2, 3;
logicalrelid | locktype | mode
-------------------------+----------+--------------------------
partitioning_locks | advisory | ShareUpdateExclusiveLock
partitioning_locks | advisory | ShareUpdateExclusiveLock
partitioning_locks | advisory | ShareUpdateExclusiveLock
partitioning_locks | advisory | ShareUpdateExclusiveLock
logicalrelid | locktype | mode
-------------------------+----------+-----------
partitioning_locks_2009 | advisory | ShareLock
partitioning_locks_2009 | advisory | ShareLock
partitioning_locks_2009 | advisory | ShareLock
partitioning_locks_2009 | advisory | ShareLock
partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock
partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock
partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock
partitioning_locks_2009 | advisory | ShareUpdateExclusiveLock
(12 rows)
(4 rows)
COMMIT;
-- test shard resource locks with INSERT/SELECT

File diff suppressed because it is too large Load Diff

View File

@ -252,7 +252,7 @@ SELECT plpgsql_test_2();
-- run the tests which do not require re-partition
-- with real-time executor
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- now, run PL/pgsql functions
SELECT plpgsql_test_1();
plpgsql_test_1
@ -875,7 +875,7 @@ SELECT task_tracker_partition_column_select(6);
(6,)
(4 rows)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- check updates
CREATE FUNCTION partition_parameter_update(int, int) RETURNS void as $$
BEGIN

View File

@ -221,7 +221,7 @@ SELECT * from prepared_sql_test_7;
(1 row)
-- now, run some of the tests with real-time executor
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- execute prepared statements
EXECUTE prepared_test_1;
count
@ -782,7 +782,7 @@ EXECUTE prepared_task_tracker_partition_column_select(6);
6 |
(4 rows)
SET citus.task_executor_type TO 'real-time';
RESET citus.task_executor_type;
-- check updates
PREPARE prepared_partition_parameter_update(int, int) AS
UPDATE prepare_table SET value = $2 WHERE key = $1;

Some files were not shown because too many files have changed in this diff Show More