From 40da78c6fd2c0c4396fe63c870fc4efc8e4c8cad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C3=96nder=20Kalac=C4=B1?= Date: Fri, 28 Jun 2019 14:04:40 +0200 Subject: [PATCH] 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 --- .circleci/config.yml | 33 + src/backend/distributed/commands/multi_copy.c | 8 +- .../distributed/commands/utility_hook.c | 17 +- src/backend/distributed/commands/vacuum.c | 5 +- .../connection/connection_management.c | 44 +- .../connection/placement_connection.c | 127 +- .../distributed/executor/adaptive_executor.c | 3450 +++++++++++++++++ .../distributed/executor/citus_custom_scan.c | 109 +- .../executor/insert_select_executor.c | 29 +- .../executor/multi_router_executor.c | 50 +- .../executor/multi_server_executor.c | 22 +- .../distributed/master/master_create_shards.c | 25 +- .../master/master_stage_protocol.c | 118 +- .../distributed/master/master_truncate.c | 10 +- .../distributed/planner/distributed_planner.c | 6 + .../planner/insert_select_planner.c | 1 + .../planner/multi_router_planner.c | 15 + src/backend/distributed/shared_library_init.c | 54 +- .../transaction/remote_transaction.c | 18 +- src/backend/distributed/utils/listutils.c | 24 + src/include/distributed/citus_custom_scan.h | 1 + .../distributed/connection_management.h | 20 + src/include/distributed/listutils.h | 1 + src/include/distributed/multi_executor.h | 13 + .../distributed/multi_physical_planner.h | 2 +- .../distributed/multi_router_executor.h | 8 + .../distributed/multi_server_executor.h | 9 +- .../distributed/placement_connection.h | 4 +- src/include/distributed/remote_transaction.h | 9 + src/test/regress/Makefile | 15 + src/test/regress/bin/normalize.sed | 25 +- src/test/regress/bin/normalized_tests.lst | 25 + .../regress/expected/adaptive_executor.out | 56 + .../expected/failure_1pc_copy_append.out | 28 +- .../expected/failure_1pc_copy_append_9.out | 321 ++ .../expected/failure_1pc_copy_hash.out | 8 +- .../failure_connection_establishment.out | 125 +- .../failure_connection_establishment_9.out | 244 ++ ...ure_create_distributed_table_non_empty.out | 9 +- ...e_create_distributed_table_non_empty_9.out | 979 +++++ .../failure_create_reference_table_9.out | 248 ++ .../regress/expected/failure_create_table.out | 23 +- .../expected/failure_create_table_9.out | 691 ++++ .../expected/failure_cte_subquery_9.out | 393 ++ src/test/regress/expected/failure_ddl.out | 6 +- src/test/regress/expected/failure_ddl_9.out | 1109 ++++++ .../failure_insert_select_pushdown_9.out | 150 + ...ailure_insert_select_via_coordinator_9.out | 229 ++ .../regress/expected/failure_multi_dml_9.out | 490 +++ .../expected/failure_multi_row_insert_9.out | 158 + .../failure_multi_shard_update_delete.out | 2 + .../failure_multi_shard_update_delete_9.out | 694 ++++ .../expected/failure_real_time_select.out | 93 +- .../expected/failure_real_time_select_9.out | 314 ++ .../regress/expected/failure_ref_tables_9.out | 99 + .../regress/expected/failure_savepoints.out | 5 + .../regress/expected/failure_savepoints_9.out | 353 ++ .../regress/expected/failure_single_mod_9.out | 128 + .../expected/failure_single_select_9.out | 238 ++ .../regress/expected/failure_truncate.out | 8 +- .../regress/expected/failure_truncate_9.out | 1299 +++++++ src/test/regress/expected/failure_vacuum.out | 13 +- .../regress/expected/failure_vacuum_0.out | 5 +- .../regress/expected/failure_vacuum_8.out | 141 + .../regress/expected/failure_vacuum_9.out | 140 + .../regress/expected/intermediate_results.out | 30 +- .../isolation_citus_dist_activity.out | 116 +- ...ut => isolation_citus_dist_activity_9.out} | 97 +- ...olation_copy_placement_vs_modification.out | 10 + .../isolation_distributed_transaction_id.out | 6 +- .../isolation_dump_global_wait_edges.out | 18 +- .../isolation_multi_shard_modify_vs_all.out | 20 +- .../isolation_replace_wait_function.out | 2 +- .../expected/limit_intermediate_size.out | 42 +- .../multi_binary_master_copy_format.out | 2 +- .../expected/multi_complex_expressions.out | 2 +- .../expected/multi_complex_expressions_0.out | 2 +- src/test/regress/expected/multi_explain.out | 4 +- src/test/regress/expected/multi_extension.out | 18 +- .../expected/multi_follower_task_tracker.out | 2 +- .../expected/multi_function_in_join.out | 75 +- .../regress/expected/multi_insert_select.out | 6 +- .../expected/multi_insert_select_9.out | 2820 ++++++++++++++ .../regress/expected/multi_modifications.out | 13 +- .../expected/multi_modifications_9.out | 1279 ++++++ .../expected/multi_modifying_xacts.out | 46 +- .../expected/multi_modifying_xacts_9.out | 1582 ++++++++ src/test/regress/expected/multi_multiuser.out | 6 +- .../regress/expected/multi_multiuser_0.out | 6 +- .../regress/expected/multi_mx_explain.out | 28 +- .../expected/multi_mx_partitioning.out | 10 +- .../expected/multi_mx_partitioning_0.out | 10 +- .../expected/multi_mx_router_planner.out | 4 +- .../multi_null_minmax_value_pruning.out | 2 +- .../multi_null_minmax_value_pruning_0.out | 12 +- .../regress/expected/multi_partitioning.out | 2 +- .../regress/expected/multi_partitioning_0.out | 20 +- .../regress/expected/multi_partitioning_9.out | 1939 +++++++++ .../regress/expected/multi_prepare_plsql.out | 4 +- .../regress/expected/multi_prepare_sql.out | 4 +- .../multi_query_directory_cleanup.out | 49 +- .../multi_query_directory_cleanup_0.out | 273 ++ .../expected/multi_real_time_transaction.out | 11 +- .../regress/expected/multi_router_planner.out | 136 +- .../multi_router_planner_fast_path.out | 53 +- .../expected/multi_select_distinct.out | 64 +- .../regress/expected/multi_sql_function.out | 2 +- .../expected/multi_transaction_recovery.out | 107 +- src/test/regress/expected/multi_upsert.out | 2 +- .../expected/sequential_modifications.out | 4 + .../regress/expected/subquery_and_cte.out | 6 +- src/test/regress/expected/with_dml.out | 5 + src/test/regress/expected/with_modifying.out | 35 +- .../regress/expected/with_transactions.out | 2 +- .../regress/expected/with_transactions_9.out | 125 + src/test/regress/failure_schedule | 1 - src/test/regress/isolation_schedule | 13 +- src/test/regress/multi_schedule | 2 +- .../multi_alter_table_statements.source | 2 +- src/test/regress/pg_regress_multi.pl | 4 + .../specs/isolation_citus_dist_activity.spec | 31 +- ...lation_copy_placement_vs_modification.spec | 1 + .../isolation_distributed_transaction_id.spec | 2 +- .../isolation_multi_shard_modify_vs_all.spec | 2 +- src/test/regress/sql/adaptive_executor.sql | 37 + .../regress/sql/failure_1pc_copy_append.sql | 7 +- .../regress/sql/failure_1pc_copy_hash.sql | 3 + .../sql/failure_connection_establishment.sql | 65 +- ...ure_create_distributed_table_non_empty.sql | 11 +- src/test/regress/sql/failure_create_table.sql | 13 +- src/test/regress/sql/failure_ddl.sql | 3 + .../sql/failure_multi_shard_update_delete.sql | 3 + .../regress/sql/failure_real_time_select.sql | 58 +- src/test/regress/sql/failure_savepoints.sql | 6 + src/test/regress/sql/failure_truncate.sql | 6 + src/test/regress/sql/failure_vacuum.sql | 8 + src/test/regress/sql/intermediate_results.sql | 25 +- .../regress/sql/limit_intermediate_size.sql | 19 +- .../sql/multi_binary_master_copy_format.sql | 2 +- .../regress/sql/multi_complex_expressions.sql | 2 +- src/test/regress/sql/multi_explain.sql | 4 +- src/test/regress/sql/multi_extension.sql | 11 +- .../sql/multi_follower_task_tracker.sql | 2 +- .../regress/sql/multi_function_in_join.sql | 61 +- src/test/regress/sql/multi_insert_select.sql | 4 +- src/test/regress/sql/multi_modifications.sql | 16 +- .../regress/sql/multi_modifying_xacts.sql | 30 +- src/test/regress/sql/multi_multiuser.sql | 6 +- src/test/regress/sql/multi_mx_explain.sql | 2 +- .../regress/sql/multi_mx_partitioning.sql | 10 +- .../regress/sql/multi_mx_router_planner.sql | 4 +- .../sql/multi_null_minmax_value_pruning.sql | 2 +- src/test/regress/sql/multi_partitioning.sql | 2 +- src/test/regress/sql/multi_prepare_plsql.sql | 4 +- src/test/regress/sql/multi_prepare_sql.sql | 4 +- .../sql/multi_real_time_transaction.sql | 9 +- src/test/regress/sql/multi_router_planner.sql | 97 +- .../sql/multi_router_planner_fast_path.sql | 41 +- src/test/regress/sql/multi_sql_function.sql | 2 +- .../sql/multi_transaction_recovery.sql | 54 +- src/test/regress/sql/multi_upsert.sql | 2 +- .../regress/sql/sequential_modifications.sql | 6 + src/test/regress/sql/subquery_and_cte.sql | 4 +- src/test/regress/sql/with_dml.sql | 7 + src/test/regress/sql/with_modifying.sql | 32 +- src/test/regress/sql/with_transactions.sql | 2 +- 166 files changed, 22054 insertions(+), 899 deletions(-) create mode 100644 src/backend/distributed/executor/adaptive_executor.c create mode 100644 src/test/regress/expected/adaptive_executor.out create mode 100644 src/test/regress/expected/failure_1pc_copy_append_9.out create mode 100644 src/test/regress/expected/failure_connection_establishment_9.out create mode 100644 src/test/regress/expected/failure_create_distributed_table_non_empty_9.out create mode 100644 src/test/regress/expected/failure_create_reference_table_9.out create mode 100644 src/test/regress/expected/failure_create_table_9.out create mode 100644 src/test/regress/expected/failure_cte_subquery_9.out create mode 100644 src/test/regress/expected/failure_ddl_9.out create mode 100644 src/test/regress/expected/failure_insert_select_pushdown_9.out create mode 100644 src/test/regress/expected/failure_insert_select_via_coordinator_9.out create mode 100644 src/test/regress/expected/failure_multi_dml_9.out create mode 100644 src/test/regress/expected/failure_multi_row_insert_9.out create mode 100644 src/test/regress/expected/failure_multi_shard_update_delete_9.out create mode 100644 src/test/regress/expected/failure_real_time_select_9.out create mode 100644 src/test/regress/expected/failure_ref_tables_9.out create mode 100644 src/test/regress/expected/failure_savepoints_9.out create mode 100644 src/test/regress/expected/failure_single_mod_9.out create mode 100644 src/test/regress/expected/failure_single_select_9.out create mode 100644 src/test/regress/expected/failure_truncate_9.out create mode 100644 src/test/regress/expected/failure_vacuum_8.out create mode 100644 src/test/regress/expected/failure_vacuum_9.out rename src/test/regress/expected/{isolation_citus_dist_activity_0.out => isolation_citus_dist_activity_9.out} (50%) create mode 100644 src/test/regress/expected/multi_insert_select_9.out create mode 100644 src/test/regress/expected/multi_modifications_9.out create mode 100644 src/test/regress/expected/multi_modifying_xacts_9.out create mode 100644 src/test/regress/expected/multi_partitioning_9.out create mode 100644 src/test/regress/expected/multi_query_directory_cleanup_0.out create mode 100644 src/test/regress/expected/with_transactions_9.out create mode 100644 src/test/regress/sql/adaptive_executor.sql diff --git a/.circleci/config.yml b/.circleci/config.yml index e1b188999..8575b2421 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -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]}} diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index 99af7f654..9bf20556e 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -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 diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index 4db633c3e..5ce7f7186 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -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) { diff --git a/src/backend/distributed/commands/vacuum.c b/src/backend/distributed/commands/vacuum.c index 9288fcc6d..8f69f769e 100644 --- a/src/backend/distributed/commands/vacuum.c +++ b/src/backend/distributed/commands/vacuum.c @@ -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++; diff --git a/src/backend/distributed/connection/connection_management.c b/src/backend/distributed/connection/connection_management.c index 297c2ce12..0e9ddd7c1 100644 --- a/src/backend/distributed/connection/connection_management.c +++ b/src/backend/distributed/connection/connection_management.c @@ -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; +} diff --git a/src/backend/distributed/connection/placement_connection.c b/src/backend/distributed/connection/placement_connection.c index 8469d49d3..a9292036d 100644 --- a/src/backend/distributed/connection/placement_connection.c +++ b/src/backend/distributed/connection/placement_connection.c @@ -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. diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c new file mode 100644 index 000000000..c529bbe03 --- /dev/null +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -0,0 +1,3450 @@ +/*------------------------------------------------------------------------- + * + * 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. + * + *------------------------------------------------------------------------- + */ + +#include "postgres.h" +#include "funcapi.h" +#include "libpq-fe.h" +#include "miscadmin.h" +#include "pgstat.h" + +#include +#include + +#include "access/xact.h" +#include "commands/dbcommands.h" +#include "distributed/citus_custom_scan.h" +#include "distributed/connection_management.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_router_executor.h" +#include "distributed/multi_server_executor.h" +#include "distributed/placement_connection.h" +#include "distributed/relation_access_tracking.h" +#include "distributed/remote_commands.h" +#include "distributed/resource_lock.h" +#include "distributed/subplan_execution.h" +#include "distributed/transaction_management.h" +#include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" +#include "lib/ilist.h" +#include "storage/fd.h" +#include "storage/latch.h" +#include "utils/int8.h" +#include "utils/memutils.h" +#include "utils/timestamp.h" + +/* + * DistributedExecution represents the execution of a distributed query + * plan. + */ +typedef struct DistributedExecution +{ + /* the corresponding distributed plan's operation */ + CmdType operation; + + List *tasksToExecute; + + /* the corresponding distributed plan has RETURNING */ + bool hasReturning; + + /* Parameters for parameterized plans. Can be NULL. */ + ParamListInfo paramListInfo; + + /* Tuple descriptor and destination for result. Can be NULL. */ + TupleDesc tupleDescriptor; + Tuplestorestate *tupleStore; + + + /* list of workers involved in the execution */ + List *workerList; + + /* list of all connections used for distributed execution */ + List *sessionList; + + /* + * Flag to indiciate that the set of connections we are interested + * in has changed and waitEventSet needs to be rebuilt. + */ + bool connectionSetChanged; + + /* + * Flag to indiciate that the set of wait events we are interested + * in might have changed and waitEventSet needs to be updated. + * + * Note that we set this flag whenever we assign a value to waitFlags, + * but we don't check that the waitFlags is actually different from the + * previous value. So we might have some false positives for this flag, + * which is OK, because in this case ModifyWaitEvent() is noop. + */ + bool waitFlagsChanged; + + /* + * WaitEventSet used for waiting for I/O events. + * + * 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. + */ + WaitEventSet *waitEventSet; + + /* + * The number of connections we aim to open per worker. + * + * If there are no more tasks to assigned, the actual number may be lower. + * If there are already more connections, the actual number may be higher. + */ + int targetPoolSize; + + /* total number of tasks to execute */ + int totalTaskCount; + + /* number of tasks that still need to be executed */ + int unfinishedTaskCount; + + /* + * Flag to indicate whether throwing errors on cancellation is + * allowed. + */ + bool raiseInterrupts; + + /* + * Flag to indicate whether the query is running in a distributed + * transaction. + */ + bool isTransaction; + + /* indicates whether distributed execution has failed */ + bool failed; + + /* set to true when we prefer to bail out early */ + bool errorOnAnyFailure; + + /* + * For SELECT commands or INSERT/UPDATE/DELETE commands with RETURNING, + * the total number of rows received from the workers. For + * INSERT/UPDATE/DELETE commands without RETURNING, the total number of + * tuples modified. + * + * Note that for replicated tables (e.g., reference tables), we only consider + * a single replica's rows that are processed. + */ + uint64 rowsProcessed; + + /* statistics on distributed execution */ + DistributedExecutionStats *executionStats; +} DistributedExecution; + +/* + * WorkerPool represents a pool of sessions on the same worker. + * + * A WorkerPool has two queues containing the TaskPlacementExecutions that need + * to be executed on the worker. + * + * TaskPlacementExecutions that are ready to execute are in readyTaskQueue. + * TaskPlacementExecutions that may need to be executed once execution on + * another worker finishes or fails are in pendingTaskQueue. + * + * In TransactionStateMachine, the sessions opportunistically take + * TaskPlacementExecutions from the readyQueue when they are ready and have no + * assigned tasks. + * + * We track connection timeouts per WorkerPool. When the first connection is + * established we set the poolStartTime and if no connection can be established + * before NodeConnectionTime, the WorkerPool fails. There is some specialised + * logic in case citus.force_max_query_parallelization is enabled because we + * may fail to establish a connection per placement after already establishing + * some connections earlier in the execution. + * + * A WorkerPool fails if all connection attempts failed or all connections + * are lost. In that case, all TaskPlacementExecutions in the queues are + * marked as failed in PlacementExecutionDone, which typically causes the + * task and therefore the distributed execution to fail. In case of a + * replicated table or a SELECT on a reference table, the remaining placements + * will be tried by moving them from a pendingTaskQueue to a readyTaskQueue. + */ +typedef struct WorkerPool +{ + /* distributed execution in which the worker participates */ + DistributedExecution *distributedExecution; + + /* worker node on which we have a pool of sessions */ + WorkerNode *node; + + /* all sessions on the worker that are part of the current execution */ + List *sessionList; + + /* number of connections that were established */ + int activeConnectionCount; + + /* + * Keep track of how many connections are ready for execution, in + * order to (efficiently) know whether more connections to the worker + * are needed. + */ + int idleConnectionCount; + + /* number of connections that did not send a command */ + int unusedConnectionCount; + + /* number of failed connections */ + int failedConnectionCount; + + /* + * Placement executions destined for worker node, but not assigned to any + * connection and not yet ready to start (depends on other placement + * executions). + */ + dlist_head pendingTaskQueue; + + /* + * Placement executions destined for worker node, but not assigned to any + * connection and not ready to start. + */ + dlist_head readyTaskQueue; + int readyTaskCount; + + /* + * We keep this for enforcing the connection timeouts. In our definition, a pool + * starts when the first connection establishment starts. + */ + TimestampTz poolStartTime; + + /* indicates whether to check for the connection timeout */ + bool checkForPoolTimeout; + + /* last time we opened a connection */ + TimestampTz lastConnectionOpenTime; + + /* maximum number of connections we are allowed to open at once */ + uint32 maxNewConnectionsPerCycle; + + /* + * This is only set in WorkerPoolFailed() function. Once a pool fails, we do not + * use it anymore. + */ + bool failed; +} WorkerPool; + +struct TaskPlacementExecution; + +/* + * WorkerSession represents a session on a worker that can execute tasks + * (sequentially) and is part of a WorkerPool. + * + * Each WorkerSession has two queues containing TaskPlacementExecutions that + * need to be executed within this particular session because the session + * accessed the same or co-located placements earlier in the transaction. + * + * TaskPlacementExecutions that are ready to execute are in readyTaskQueue. + * TaskPlacementExecutions that may need to be executed once execution on + * another worker finishes or fails are in pendingTaskQueue. + */ +typedef struct WorkerSession +{ + /* only useful for debugging */ + uint64 sessionId; + + /* worker pool of which this session is part */ + WorkerPool *workerPool; + + /* connection over which the session is established */ + MultiConnection *connection; + + /* tasks that need to be executed on this connection, but are not ready to start */ + dlist_head pendingTaskQueue; + + /* tasks that need to be executed on this connection and are ready to start */ + dlist_head readyTaskQueue; + + /* task the worker should work on or NULL */ + struct TaskPlacementExecution *currentTask; + + /* + * The number of commands sent to the worker over the session. Excludes + * distributed transaction related commands such as BEGIN/COMMIT etc. + */ + uint64 commandsSent; +} WorkerSession; + + +struct TaskPlacementExecution; + + +/* + * TaskExecutionState indicates whether or not a command on a shard + * has finished, or whether it has failed. + */ +typedef enum TaskExecutionState +{ + TASK_EXECUTION_NOT_FINISHED, + TASK_EXECUTION_FINISHED, + TASK_EXECUTION_FAILED +} TaskExecutionState; + +/* + * PlacementExecutionOrder indicates whether a command should be executed + * on any replica, on all replicas sequentially (in order), or on all + * replicas in parallel. + */ +typedef enum PlacementExecutionOrder +{ + EXECUTION_ORDER_ANY, + EXECUTION_ORDER_SEQUENTIAL, + EXECUTION_ORDER_PARALLEL, +} PlacementExecutionOrder; + + +/* + * ShardCommandExecution represents an execution of a command on a shard + * that may (need to) run across multiple placements. + */ +typedef struct ShardCommandExecution +{ + /* description of the task */ + Task *task; + + /* order in which the command should be replicated on replicas */ + PlacementExecutionOrder executionOrder; + + /* executions of the command on the placements of the shard */ + struct TaskPlacementExecution **placementExecutions; + int placementExecutionCount; + + /* whether we expect results to come back */ + bool expectResults; + + /* + * RETURNING results from other shard placements can be ignored + * after we got results from the first placements. + */ + bool gotResults; + + TaskExecutionState executionState; +} ShardCommandExecution; + +/* + * TaskPlacementExecutionState indicates whether a command is running + * on a shard placement, or finished or failed. + */ +typedef enum TaskPlacementExecutionState +{ + PLACEMENT_EXECUTION_NOT_READY, + PLACEMENT_EXECUTION_READY, + PLACEMENT_EXECUTION_RUNNING, + PLACEMENT_EXECUTION_FINISHED, + PLACEMENT_EXECUTION_FAILED +} TaskPlacementExecutionState; + +/* + * TaskPlacementExecution represents the an execution of a command + * on a shard placement. + */ +typedef struct TaskPlacementExecution +{ + /* shard command execution of which this placement execution is part */ + ShardCommandExecution *shardCommandExecution; + + /* shard placement on which this command runs */ + ShardPlacement *shardPlacement; + + /* state of the execution of the command on the placement */ + TaskPlacementExecutionState executionState; + + /* worker pool on which the placement needs to be executed */ + WorkerPool *workerPool; + + /* the session the placement execution is assigned to or NULL */ + WorkerSession *assignedSession; + + /* membership in assigned task queue of a particular session */ + dlist_node sessionPendingQueueNode; + + /* membership in ready-to-start assigned task queue of a particular session */ + dlist_node sessionReadyQueueNode; + + /* membership in assigned task queue of worker */ + dlist_node workerPendingQueueNode; + + /* membership in ready-to-start task queue of worker */ + dlist_node workerReadyQueueNode; + + /* index in array of placement executions in a ShardCommandExecution */ + int placementExecutionIndex; +} TaskPlacementExecution; + + +/* GUC, determining whether Citus opens 1 connection per task */ +bool ForceMaxQueryParallelization = false; +int MaxAdaptiveExecutorPoolSize = 16; + +/* GUC, number of ms to wait between opening connections to the same worker */ +int ExecutorSlowStartInterval = 10; + + +/* local functions */ +static DistributedExecution * CreateDistributedExecution(CmdType operation, + List *taskList, + bool hasReturning, + ParamListInfo paramListInfo, + TupleDesc tupleDescriptor, + Tuplestorestate *tupleStore, + int targetPoolSize); +static void StartDistributedExecution(DistributedExecution *execution); +static void RunDistributedExecution(DistributedExecution *execution); +static bool ShouldRunTasksSequentially(List *taskList); +static void SequentialRunDistributedExecution(DistributedExecution *execution); + +static void FinishDistributedExecution(DistributedExecution *execution); +static void CleanUpSessions(DistributedExecution *execution); + +static void LockPartitionsForDistributedPlan(DistributedPlan *distributedPlan); +static void AcquireExecutorShardLocks(DistributedExecution *execution); +static bool DistributedExecutionModifiesDatabase(DistributedExecution *execution); +static bool DistributedPlanModifiesDatabase(DistributedPlan *plan); +static bool TaskListModifiesDatabase(CmdType operation, List *taskList); +static bool DistributedExecutionRequiresRollback(DistributedExecution *execution); +static bool SelectForUpdateOnReferenceTable(CmdType operation, List *taskList); +static void AssignTasksToConnections(DistributedExecution *execution); +static void UnclaimAllSessionConnections(List *sessionList); +static bool UseConnectionPerPlacement(void); +static PlacementExecutionOrder ExecutionOrderForTask(CmdType operation, Task *task); +static WorkerPool * FindOrCreateWorkerPool(DistributedExecution *execution, + WorkerNode *workerNode); +static WorkerSession * FindOrCreateWorkerSession(WorkerPool *workerPool, + MultiConnection *connection); +static void ManageWorkerPool(WorkerPool *workerPool); +static void CheckConnectionTimeout(WorkerPool *workerPool); +static int UsableConnectionCount(WorkerPool *workerPool); +static long NextEventTimeout(DistributedExecution *execution); +static long MillisecondsBetweenTimestamps(TimestampTz startTime, TimestampTz endTime); +static WaitEventSet * BuildWaitEventSet(List *sessionList); +static TaskPlacementExecution * PopPlacementExecution(WorkerSession *session); +static TaskPlacementExecution * PopAssignedPlacementExecution(WorkerSession *session); +static TaskPlacementExecution * PopUnassignedPlacementExecution(WorkerPool *workerPool); +static bool StartPlacementExecutionOnSession(TaskPlacementExecution *placementExecution, + WorkerSession *session); +static List * PlacementAccessListForTask(Task *task, ShardPlacement *taskPlacement); +static void ConnectionStateMachine(WorkerSession *session); +static void Activate2PCIfModifyingTransactionExpandsToNewNode(WorkerSession *session); +static bool TransactionModifiedDistributedTable(DistributedExecution *execution); +static void TransactionStateMachine(WorkerSession *session); +static void UpdateConnectionWaitFlags(WorkerSession *session, int waitFlags); +static bool CheckConnectionReady(WorkerSession *session); +static bool ReceiveResults(WorkerSession *session, bool storeRows); +static void WorkerSessionFailed(WorkerSession *session); +static void WorkerPoolFailed(WorkerPool *workerPool); +static void PlacementExecutionDone(TaskPlacementExecution *placementExecution, + bool succeeded); +static void ScheduleNextPlacementExecution(TaskPlacementExecution *placementExecution, + bool succeeded); +static bool ShouldMarkPlacementsInvalidOnFailure(DistributedExecution *execution); +static void PlacementExecutionReady(TaskPlacementExecution *placementExecution); +static TaskExecutionState TaskExecutionStateMachine(ShardCommandExecution * + shardCommandExecution); + + +/* + * AdaptiveExecutor is called via CitusExecScan on the + * first call of CitusExecScan. The function fills the tupleStore + * of the input scanScate. + */ +TupleTableSlot * +AdaptiveExecutor(CustomScanState *node) +{ + CitusScanState *scanState = (CitusScanState *) node; + TupleTableSlot *resultSlot = NULL; + + DistributedPlan *distributedPlan = scanState->distributedPlan; + DistributedExecution *execution = NULL; + EState *executorState = ScanStateGetExecutorState(scanState); + ParamListInfo paramListInfo = executorState->es_param_list_info; + Tuplestorestate *tupleStore = NULL; + TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(scanState); + bool randomAccess = true; + bool interTransactions = false; + int targetPoolSize = MaxAdaptiveExecutorPoolSize; + + Job *job = distributedPlan->workerJob; + List *taskList = job->taskList; + + /* we should only call this once before the scan finished */ + Assert(!scanState->finishedRemoteScan); + + /* + * PostgreSQL takes locks on all partitions in the executor. It's not entirely + * clear why this is necessary (instead of locking the parent during DDL), but + * We do the same for consistency. + */ + LockPartitionsForDistributedPlan(distributedPlan); + + ExecuteSubPlans(distributedPlan); + + scanState->tuplestorestate = + tuplestore_begin_heap(randomAccess, interTransactions, work_mem); + tupleStore = scanState->tuplestorestate; + + if (MultiShardConnectionType == SEQUENTIAL_CONNECTION) + { + targetPoolSize = 1; + } + + execution = CreateDistributedExecution(distributedPlan->operation, taskList, + distributedPlan->hasReturning, + paramListInfo, tupleDescriptor, + tupleStore, targetPoolSize); + + StartDistributedExecution(execution); + + if (ShouldRunTasksSequentially(execution->tasksToExecute)) + { + SequentialRunDistributedExecution(execution); + } + else + { + RunDistributedExecution(execution); + } + + if (distributedPlan->operation != CMD_SELECT) + { + executorState->es_processed = execution->rowsProcessed; + } + + FinishDistributedExecution(execution); + + if (SortReturning && distributedPlan->hasReturning) + { + SortTupleStore(scanState); + } + + return resultSlot; +} + + +/* + * ExecuteUtilityTaskListWithoutResults is a wrapper around executing task + * list for utility commands. If the adaptive executor is enabled, the function + * executes the task list via the adaptive executor. Else, the function goes + * through router executor. + */ +void +ExecuteUtilityTaskListWithoutResults(List *taskList, int targetPoolSize, + bool forceSequentialExecution) +{ + if (TaskExecutorType == MULTI_EXECUTOR_ADAPTIVE) + { + ExecuteTaskList(CMD_UTILITY, taskList, targetPoolSize); + } + else + { + if (MultiShardConnectionType == SEQUENTIAL_CONNECTION || + forceSequentialExecution) + { + ExecuteModifyTasksSequentiallyWithoutResults(taskList, CMD_UTILITY); + } + else + { + ExecuteModifyTasksWithoutResults(taskList); + } + } +} + + +/* + * ExecuteTaskList is a proxy to ExecuteTaskListExtended() with defaults + * for some of the arguments. + */ +uint64 +ExecuteTaskList(CmdType operation, List *taskList, int targetPoolSize) +{ + TupleDesc tupleDescriptor = NULL; + Tuplestorestate *tupleStore = NULL; + bool hasReturning = false; + + return ExecuteTaskListExtended(operation, taskList, tupleDescriptor, + tupleStore, hasReturning, targetPoolSize); +} + + +/* + * ExecuteTaskListExtended sets up the execution for given task list and + * runs it. + */ +uint64 +ExecuteTaskListExtended(CmdType operation, List *taskList, + TupleDesc tupleDescriptor, Tuplestorestate *tupleStore, + bool hasReturning, int targetPoolSize) +{ + DistributedExecution *execution = NULL; + ParamListInfo paramListInfo = NULL; + + if (MultiShardConnectionType == SEQUENTIAL_CONNECTION) + { + targetPoolSize = 1; + } + + execution = + CreateDistributedExecution(operation, taskList, hasReturning, paramListInfo, + tupleDescriptor, tupleStore, targetPoolSize); + + StartDistributedExecution(execution); + RunDistributedExecution(execution); + FinishDistributedExecution(execution); + + return execution->rowsProcessed; +} + + +/* + * CreateDistributedExecution creates a distributed execution data structure for + * a distributed plan. + */ +DistributedExecution * +CreateDistributedExecution(CmdType operation, List *taskList, bool hasReturning, + ParamListInfo paramListInfo, TupleDesc tupleDescriptor, + Tuplestorestate *tupleStore, int targetPoolSize) +{ + DistributedExecution *execution = + (DistributedExecution *) palloc0(sizeof(DistributedExecution)); + + execution->operation = operation; + execution->tasksToExecute = taskList; + execution->hasReturning = hasReturning; + + execution->executionStats = + (DistributedExecutionStats *) palloc0(sizeof(DistributedExecutionStats)); + execution->paramListInfo = paramListInfo; + execution->tupleDescriptor = tupleDescriptor; + execution->tupleStore = tupleStore; + + execution->workerList = NIL; + execution->sessionList = NIL; + execution->targetPoolSize = targetPoolSize; + + execution->totalTaskCount = list_length(taskList); + execution->unfinishedTaskCount = list_length(taskList); + execution->rowsProcessed = 0; + + execution->raiseInterrupts = true; + + execution->connectionSetChanged = false; + execution->waitFlagsChanged = false; + + return execution; +} + + +/* + * StartDistributedExecution sets up the coordinated transaction and 2PC for + * the execution whenever necessary. It also keeps track of parallel relation + * accesses to enforce restrictions that arise due to foreign keys to reference + * tables. + */ +void +StartDistributedExecution(DistributedExecution *execution) +{ + List *taskList = execution->tasksToExecute; + + if (MultiShardCommitProtocol != COMMIT_PROTOCOL_BARE) + { + if (DistributedExecutionRequiresRollback(execution)) + { + BeginOrContinueCoordinatedTransaction(); + + if (TaskListRequires2PC(taskList)) + { + /* + * Although using two phase commit protocol is an independent decision than + * failing on any error, we prefer to couple them. Our motivation is that + * the failures are rare, and we prefer to avoid marking placements invalid + * in case of failures. + */ + CoordinatedTransactionUse2PC(); + + execution->errorOnAnyFailure = true; + } + else if (MultiShardCommitProtocol != COMMIT_PROTOCOL_2PC && + list_length(taskList) > 1 && + DistributedExecutionModifiesDatabase(execution)) + { + /* + * Even if we're not using 2PC, we prefer to error out + * on any failures during multi shard modifications/DDLs. + */ + execution->errorOnAnyFailure = true; + } + } + } + else + { + /* + * We prefer to error on any failures for CREATE INDEX + * CONCURRENTLY or VACUUM//VACUUM ANALYZE (e.g., COMMIT_PROTOCOL_BARE). + */ + execution->errorOnAnyFailure = true; + } + + execution->isTransaction = InCoordinatedTransaction(); + + /* + * We should not record parallel access if the target pool size is less than 2. + * The reason is that we define parallel access as at least two connections + * accessing established to worker node. + * + * It is not ideal to have this check here, it'd have been better if we simply passed + * DistributedExecution directly to the RecordParallelAccess*() function. However, + * since we have two other executors that rely on the function, we had to only pass + * the tasklist to have a common API. + */ + if (execution->targetPoolSize > 1) + { + RecordParallelRelationAccessForTaskList(taskList); + } + + /* prevent unsafe concurrent modifications */ + AcquireExecutorShardLocks(execution); +} + + +/* + * DistributedExecutionModifiesDatabase returns true if the execution modifies the data + * or the schema. + */ +static bool +DistributedExecutionModifiesDatabase(DistributedExecution *execution) +{ + return TaskListModifiesDatabase(execution->operation, execution->tasksToExecute); +} + + +/* + * DistributedPlanModifiesDatabase returns true if the plan modifies the data + * or the schema. + */ +static bool +DistributedPlanModifiesDatabase(DistributedPlan *plan) +{ + return TaskListModifiesDatabase(plan->operation, plan->workerJob->taskList); +} + + +/* + * TaskListModifiesDatabase is a helper function for DistributedExecutionModifiesDatabase and + * DistributedPlanModifiesDatabase. + */ +static bool +TaskListModifiesDatabase(CmdType operation, List *taskList) +{ + Task *firstTask = NULL; + + if (operation == CMD_INSERT || operation == CMD_UPDATE || operation == CMD_DELETE) + { + return true; + } + + /* + * If we cannot decide by only checking the operation, we should look closer + * to the tasks. + */ + if (list_length(taskList) < 1) + { + /* does this ever possible? */ + return false; + } + + firstTask = (Task *) linitial(taskList); + + return !ReadOnlyTask(firstTask->taskType); +} + + +/* + * DistributedExecutionRequiresRollback returns true if the distributed + * execution should start a CoordinatedTransaction. In other words, if the + * function returns true, the execution sends BEGIN; to every connection + * involved in the distributed execution. + */ +static bool +DistributedExecutionRequiresRollback(DistributedExecution *execution) +{ + List *taskList = execution->tasksToExecute; + int taskCount = list_length(taskList); + Task *task = NULL; + bool selectForUpdate = false; + + if (MultiShardCommitProtocol == COMMIT_PROTOCOL_BARE) + { + return false; + } + + if (taskCount == 0) + { + return false; + } + + task = (Task *) linitial(taskList); + + selectForUpdate = task->relationRowLockList != NIL; + if (selectForUpdate) + { + /* + * Do not check SelectOpensTransactionBlock, always open transaction block + * if SELECT FOR UPDATE is executed inside a distributed transaction. + */ + return IsTransactionBlock(); + } + + if (ReadOnlyTask(task->taskType)) + { + return SelectOpensTransactionBlock && IsTransactionBlock(); + } + + if (IsMultiStatementTransaction()) + { + return true; + } + + if (list_length(taskList) > 1) + { + return true; + } + + /* + * Checking the first task's placement list is not sufficient for all purposes since + * for append/range distributed tables we might have unequal number of placements for + * shards. However, it is safe to do here, because we're searching for a reference + * table. All other cases return false for this purpose. + */ + task = (Task *) linitial(taskList); + if (list_length(task->taskPlacementList) > 1) + { + /* + * Some tasks don't set replicationModel thus we only + * rely on the anchorShardId, not replicationModel. + * + * TODO: Do we ever need replicationModel in the Task structure? + * Can't we always rely on anchorShardId? + */ + uint64 anchorShardId = task->anchorShardId; + if (anchorShardId != INVALID_SHARD_ID && ReferenceTableShardId(anchorShardId)) + { + return true; + } + + /* + * Single DML/DDL tasks with replicated tables (non-reference) + * should not require BEGIN/COMMIT/ROLLBACK. + */ + return false; + } + + return false; +} + + +/* + * SelectForUpdateOnReferenceTable returns true if the input task + * that contains FOR UPDATE clause that locks any reference tables. + */ +static bool +SelectForUpdateOnReferenceTable(CmdType operation, List *taskList) +{ + Task *task = NULL; + ListCell *rtiLockCell = NULL; + + if (operation != CMD_SELECT) + { + return false; + } + + if (list_length(taskList) != 1) + { + /* we currently do not support SELECT FOR UPDATE on multi task queries */ + return false; + } + + task = (Task *) linitial(taskList); + foreach(rtiLockCell, task->relationRowLockList) + { + RelationRowLock *relationRowLock = (RelationRowLock *) lfirst(rtiLockCell); + Oid relationId = relationRowLock->relationId; + + if (PartitionMethod(relationId) == DISTRIBUTE_BY_NONE) + { + return true; + } + } + + return false; +} + + +/* + * LockPartitionsForDistributedPlan ensures commands take locks on all partitions + * of a distributed table that appears in the query. We do this primarily out of + * consistency with PostgreSQL locking. + */ +static void +LockPartitionsForDistributedPlan(DistributedPlan *distributedPlan) +{ + if (DistributedPlanModifiesDatabase(distributedPlan)) + { + Oid targetRelationId = distributedPlan->targetRelationId; + + LockPartitionsInRelationList(list_make1_oid(targetRelationId), RowExclusiveLock); + } + + /* + * Lock partitions of tables that appear in a SELECT or subquery. In the + * DML case this also includes the target relation, but since we already + * have a stronger lock this doesn't do any harm. + */ + LockPartitionsInRelationList(distributedPlan->relationIdList, AccessShareLock); +} + + +/* + * AcquireExecutorShardLocks acquires advisory lock on shard IDs to prevent + * unsafe concurrent modifications of shards. + * + * We prevent concurrent modifications of shards in two cases: + * 1. Any non-commutative writes to a replicated table + * 2. Multi-shard writes that are executed in parallel + * + * The first case ensures we do not apply updates in different orders on + * different replicas (e.g. of a reference table), which could lead the + * replicas to diverge. + * + * The second case prevents deadlocks due to out-of-order execution. + * + * We do not take executor shard locks for utility commands such as + * TRUNCATE because the table locks already prevent concurrent access. + */ +static void +AcquireExecutorShardLocks(DistributedExecution *execution) +{ + CmdType operation = execution->operation; + List *taskList = execution->tasksToExecute; + + if (!(operation == CMD_INSERT || operation == CMD_UPDATE || operation == CMD_DELETE || + SelectForUpdateOnReferenceTable(operation, taskList))) + { + /* + * Executor locks only apply to DML commands and SELECT FOR UPDATE queries + * touching reference tables. + */ + return; + } + + /* + * When executing in sequential mode or only executing a single task, we + * do not need multi-shard locks. + */ + if (list_length(taskList) == 1 || ShouldRunTasksSequentially(taskList)) + { + ListCell *taskCell = NULL; + + foreach(taskCell, taskList) + { + Task *task = (Task *) lfirst(taskCell); + + AcquireExecutorShardLock(task, operation); + } + } + else if (list_length(taskList) > 1) + { + AcquireExecutorMultiShardLocks(taskList); + } +} + + +/* + * FinishDistributedExecution cleans up resources associated with a + * distributed execution. In particular, it releases connections and + * clears their state. + */ +static void +FinishDistributedExecution(DistributedExecution *execution) +{ + UnsetCitusNoticeLevel(); + + if (DistributedExecutionModifiesDatabase(execution)) + { + /* prevent copying shards in same transaction */ + XactModificationLevel = XACT_MODIFICATION_DATA; + } +} + + +/* + * CleanUpSessions does any clean-up necessary for the session + * used during the execution. We only reach the function after + * successfully completing all the tasks and we expect no tasks + * are still in progress. + */ +static void +CleanUpSessions(DistributedExecution *execution) +{ + List *sessionList = execution->sessionList; + ListCell *sessionCell = NULL; + + /* we get to this function only after successful executions */ + Assert(!execution->failed && execution->unfinishedTaskCount == 0); + + /* always trigger wait event set in the first round */ + foreach(sessionCell, sessionList) + { + WorkerSession *session = lfirst(sessionCell); + + MultiConnection *connection = session->connection; + + elog(DEBUG4, "Total number of commands sent over the session %ld: %ld", + session->sessionId, session->commandsSent); + + UnclaimConnection(connection); + + if (connection->connectionState == MULTI_CONNECTION_CONNECTING || + connection->connectionState == MULTI_CONNECTION_FAILED || + connection->connectionState == MULTI_CONNECTION_LOST) + { + /* + * We want the MultiConnection go away and not used in + * the subsequent executions. + * + * We cannot get MULTI_CONNECTION_LOST via the ConnectionStateMachine, + * but we might get it via the connection API and find us here before + * changing any states in the ConnectionStateMachine. + */ + CloseConnection(connection); + } + else if (connection->connectionState == MULTI_CONNECTION_CONNECTED) + { + RemoteTransaction *transaction = &(connection->remoteTransaction); + RemoteTransactionState transactionState = transaction->transactionState; + + if (transactionState == REMOTE_TRANS_CLEARING_RESULTS) + { + /* + * We might have established the connection, and even sent BEGIN, but not + * get to the point where we assigned a task to this specific connection + * (because other connections in the pool already finished all the tasks). + */ + Assert(session->commandsSent == 0); + + ClearResults(connection, false); + } + else if (!(transactionState == REMOTE_TRANS_INVALID || + transactionState == REMOTE_TRANS_STARTED)) + { + /* + * We don't have to handle anything else. Note that the execution + * could only finish on connectionStates of MULTI_CONNECTION_CONNECTING, + * MULTI_CONNECTION_FAILED and MULTI_CONNECTION_CONNECTED. The first two + * are already handled above. + * + * When we're on MULTI_CONNECTION_CONNECTED, TransactionStateMachine + * ensures that all the necessary commands are successfully sent over + * the connection and everything is cleared up. Otherwise, we'd have been + * on MULTI_CONNECTION_FAILED state. + */ + elog(WARNING, "unexpected transaction state at the end of execution: %d", + transactionState); + } + } + else + { + elog(WARNING, "unexpected connection state at the end of execution: %d", + connection->connectionState); + } + + /* get ready for the next executions if we need use the same connection */ + connection->waitFlags = WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE; + execution->waitFlagsChanged = true; + } +} + + +/* + * UnclaimAllSessionConnections unclaims all of the connections for the given + * sessionList. + */ +static void +UnclaimAllSessionConnections(List *sessionList) +{ + ListCell *sessionCell = NULL; + foreach(sessionCell, sessionList) + { + WorkerSession *session = lfirst(sessionCell); + MultiConnection *connection = session->connection; + + UnclaimConnection(connection); + } +} + + +/* + * AssignTasksToConnections goes through the list of tasks to determine whether any + * task placements need to be assigned to particular connections because of preceding + * operations in the transaction. It then adds those connections to the pool and adds + * the task placement executions to the assigned task queue of the connection. + */ +static void +AssignTasksToConnections(DistributedExecution *execution) +{ + CmdType operation = execution->operation; + List *taskList = execution->tasksToExecute; + bool hasReturning = execution->hasReturning; + + ListCell *taskCell = NULL; + ListCell *sessionCell = NULL; + + foreach(taskCell, taskList) + { + Task *task = (Task *) lfirst(taskCell); + ShardCommandExecution *shardCommandExecution = NULL; + ListCell *taskPlacementCell = NULL; + bool placementExecutionReady = true; + int placementExecutionIndex = 0; + int placementExecutionCount = list_length(task->taskPlacementList); + + /* + * Execution of a command on a shard, which may have multiple replicas. + */ + shardCommandExecution = + (ShardCommandExecution *) palloc0(sizeof(ShardCommandExecution)); + shardCommandExecution->task = task; + shardCommandExecution->executionOrder = ExecutionOrderForTask(operation, task); + shardCommandExecution->executionState = TASK_EXECUTION_NOT_FINISHED; + shardCommandExecution->placementExecutions = + (TaskPlacementExecution **) palloc0(placementExecutionCount * + sizeof(TaskPlacementExecution *)); + shardCommandExecution->placementExecutionCount = placementExecutionCount; + shardCommandExecution->expectResults = hasReturning || operation == CMD_SELECT; + + + foreach(taskPlacementCell, task->taskPlacementList) + { + ShardPlacement *taskPlacement = (ShardPlacement *) lfirst(taskPlacementCell); + List *placementAccessList = NULL; + MultiConnection *connection = NULL; + int connectionFlags = 0; + TaskPlacementExecution *placementExecution = NULL; + WorkerNode *node = FindWorkerNode(taskPlacement->nodeName, + taskPlacement->nodePort); + WorkerPool *workerPool = FindOrCreateWorkerPool(execution, node); + + /* + * Execution of a command on a shard placement, which may not always + * happen if the query is read-only and the shard has multiple placements. + */ + placementExecution = + (TaskPlacementExecution *) palloc0(sizeof(TaskPlacementExecution)); + placementExecution->shardCommandExecution = shardCommandExecution; + placementExecution->shardPlacement = taskPlacement; + placementExecution->workerPool = workerPool; + placementExecution->placementExecutionIndex = placementExecutionIndex; + + if (placementExecutionReady) + { + placementExecution->executionState = PLACEMENT_EXECUTION_READY; + } + else + { + placementExecution->executionState = PLACEMENT_EXECUTION_NOT_READY; + } + + shardCommandExecution->placementExecutions[placementExecutionIndex] = + placementExecution; + + placementExecutionIndex++; + + placementAccessList = PlacementAccessListForTask(task, taskPlacement); + + /* + * Determine whether the task has to be assigned to a particular connection + * due to a preceding access to the placement in the same transaction. + */ + connection = GetConnectionIfPlacementAccessedInXact(connectionFlags, + placementAccessList, + NULL); + if (connection != NULL) + { + /* + * Note: We may get the same connection for multiple task placements. + * FindOrCreateWorkerSession ensures that we only have one session per + * connection. + */ + WorkerSession *session = + FindOrCreateWorkerSession(workerPool, connection); + + elog(DEBUG4, "Session %ld (%s:%d) has an assigned task", + session->sessionId, connection->hostname, connection->port); + + placementExecution->assignedSession = session; + + /* if executed, this task placement must use this session */ + if (placementExecutionReady) + { + dlist_push_tail(&session->readyTaskQueue, + &placementExecution->sessionReadyQueueNode); + } + else + { + dlist_push_tail(&session->pendingTaskQueue, + &placementExecution->sessionPendingQueueNode); + } + + /* always poll the connection in the first round */ + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + + /* If the connections are already avaliable, make sure to activate + * 2PC when necessary. + */ + Activate2PCIfModifyingTransactionExpandsToNewNode(session); + } + else + { + placementExecution->assignedSession = NULL; + + if (placementExecutionReady) + { + /* task is ready to execute on any session */ + dlist_push_tail(&workerPool->readyTaskQueue, + &placementExecution->workerReadyQueueNode); + + workerPool->readyTaskCount++; + } + else + { + /* task can be executed on any session, but is not yet ready */ + dlist_push_tail(&workerPool->pendingTaskQueue, + &placementExecution->workerPendingQueueNode); + } + } + + if (shardCommandExecution->executionOrder != EXECUTION_ORDER_PARALLEL) + { + /* + * Except for commands that can be executed across all placements + * in parallel, only the first placement execution is immediately + * ready. Set placementExecutionReady to false for the remaining + * placements. + */ + placementExecutionReady = false; + } + } + } + + /* + * The executor claims connections exclusively to make sure that calls to + * StartNodeUserDatabaseConnection do not return the same connections. + * + * We need to do this after assigning tasks to connections because the same + * connection may be be returned multiple times by GetPlacementListConnectionIfCached. + */ + foreach(sessionCell, execution->sessionList) + { + WorkerSession *session = lfirst(sessionCell); + MultiConnection *connection = session->connection; + + ClaimConnectionExclusively(connection); + } +} + + +/* + * UseConnectionPerPlacement returns whether we should use a separate connection + * per placement even if another connection is idle. We mostly use this in testing + * scenarios. + */ +static bool +UseConnectionPerPlacement(void) +{ + return ForceMaxQueryParallelization && + MultiShardConnectionType != SEQUENTIAL_CONNECTION; +} + + +/* + * ExecutionOrderForTask gives the appropriate execution order for a task. + */ +static PlacementExecutionOrder +ExecutionOrderForTask(CmdType operation, Task *task) +{ + switch (task->taskType) + { + case SQL_TASK: + case ROUTER_TASK: + { + return EXECUTION_ORDER_ANY; + } + + case MODIFY_TASK: + { + if (operation == CMD_INSERT && !task->upsertQuery) + { + return EXECUTION_ORDER_SEQUENTIAL; + } + else + { + return EXECUTION_ORDER_PARALLEL; + } + } + + case DDL_TASK: + case VACUUM_ANALYZE_TASK: + { + return EXECUTION_ORDER_PARALLEL; + } + + case MAP_TASK: + case MERGE_TASK: + case MAP_OUTPUT_FETCH_TASK: + case MERGE_FETCH_TASK: + default: + { + elog(ERROR, "unsupported task type %d in adaptive executor", task->taskType); + } + } +} + + +/* + * FindOrCreateWorkerPool gets the pool of connections for a particular worker. + */ +static WorkerPool * +FindOrCreateWorkerPool(DistributedExecution *execution, WorkerNode *workerNode) +{ + WorkerPool *workerPool = NULL; + ListCell *workerCell = NULL; + int nodeConnectionCount = 0; + + foreach(workerCell, execution->workerList) + { + workerPool = lfirst(workerCell); + + if (WorkerNodeCompare(workerPool->node, workerNode, 0) == 0) + { + return workerPool; + } + } + + workerPool = (WorkerPool *) palloc0(sizeof(WorkerPool)); + workerPool->node = workerNode; + workerPool->poolStartTime = 0; + workerPool->distributedExecution = execution; + + /* "open" connections aggressively when there are cached connections */ + nodeConnectionCount = NodeConnectionCount(workerNode->workerName, + workerNode->workerPort); + workerPool->maxNewConnectionsPerCycle = Max(1, nodeConnectionCount); + + dlist_init(&workerPool->pendingTaskQueue); + dlist_init(&workerPool->readyTaskQueue); + + execution->workerList = lappend(execution->workerList, workerPool); + + return workerPool; +} + + +/* + * FindOrCreateWorkerSession returns a session with the given connection, + * either existing or new. New sessions are added to the worker pool and + * the distributed execution. + */ +static WorkerSession * +FindOrCreateWorkerSession(WorkerPool *workerPool, MultiConnection *connection) +{ + DistributedExecution *execution = workerPool->distributedExecution; + WorkerSession *session = NULL; + ListCell *sessionCell = NULL; + static uint64 sessionId = 1; + + foreach(sessionCell, workerPool->sessionList) + { + session = lfirst(sessionCell); + + if (session->connection == connection) + { + return session; + } + } + + session = (WorkerSession *) palloc0(sizeof(WorkerSession)); + session->sessionId = sessionId++; + session->connection = connection; + session->workerPool = workerPool; + session->commandsSent = 0; + dlist_init(&session->pendingTaskQueue); + dlist_init(&session->readyTaskQueue); + + /* keep track of how many connections are ready */ + if (connection->connectionState == MULTI_CONNECTION_CONNECTED) + { + workerPool->activeConnectionCount++; + workerPool->idleConnectionCount++; + } + + workerPool->unusedConnectionCount++; + + /* + * Record the first connection establishment time to the pool. We need this + * to enforce NodeConnectionTimeout. + */ + if (list_length(workerPool->sessionList) == 0) + { + workerPool->poolStartTime = GetCurrentTimestamp(); + workerPool->checkForPoolTimeout = true; + } + + workerPool->sessionList = lappend(workerPool->sessionList, session); + execution->sessionList = lappend(execution->sessionList, session); + + return session; +} + + +/* + * ShouldRunTasksSequentially returns true if each of the individual tasks + * should be executed one by one. Note that this is different than + * MultiShardConnectionType == SEQUENTIAL_CONNECTION case. 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. + */ +static bool +ShouldRunTasksSequentially(List *taskList) +{ + Task *initialTask = NULL; + + if (list_length(taskList) < 2) + { + /* single task plans are already qualified as sequential by definition */ + return false; + } + + /* all the tasks are the same, so we only look one */ + initialTask = (Task *) linitial(taskList); + if (initialTask->rowValuesLists != NIL) + { + /* found a multi-row INSERT */ + return true; + } + + return false; +} + + +/* + * SequentialRunDistributedExecution gets a distributed execution and + * executes each individual task in the exection sequentially, one + * task at a time. See related function ShouldRunTasksSequentially() + * for more detail on the definition of SequentialRun. + */ +static void +SequentialRunDistributedExecution(DistributedExecution *execution) +{ + List *taskList = execution->tasksToExecute; + + ListCell *taskCell = NULL; + int connectionMode = MultiShardConnectionType; + + /* + * There are some implicit assumptions about this setting for the sequential + * executions, so make sure to set it. + */ + MultiShardConnectionType = SEQUENTIAL_CONNECTION; + + foreach(taskCell, taskList) + { + Task *taskToExecute = (Task *) lfirst(taskCell); + + /* execute each task one by one */ + execution->tasksToExecute = list_make1(taskToExecute); + execution->totalTaskCount = 1; + execution->unfinishedTaskCount = 1; + + /* simply call the regular execution function */ + RunDistributedExecution(execution); + } + + /* set back the original execution mode */ + MultiShardConnectionType = connectionMode; +} + + +/* + * RunDistributedExecution runs a distributed execution to completion. It first opens + * connections for distributed execution and assigns each task with shard placements + * that have previously been modified in the current transaction to the connection + * that modified them. Then, it creates a wait event set to listen for events on + * any of the connections and runs the connection state machine when a connection + * has an event. + */ +void +RunDistributedExecution(DistributedExecution *execution) +{ + WaitEvent *events = NULL; + + AssignTasksToConnections(execution); + + PG_TRY(); + { + bool cancellationReceived = false; + + /* additional 2 is for postmaster and latch */ + int eventSetSize = list_length(execution->sessionList) + 2; + + execution->waitEventSet = BuildWaitEventSet(execution->sessionList); + events = palloc0(eventSetSize * sizeof(WaitEvent)); + + while (execution->unfinishedTaskCount > 0 && !cancellationReceived) + { + int eventCount = 0; + int eventIndex = 0; + ListCell *workerCell = NULL; + long timeout = NextEventTimeout(execution); + + foreach(workerCell, execution->workerList) + { + WorkerPool *workerPool = lfirst(workerCell); + ManageWorkerPool(workerPool); + } + + if (execution->connectionSetChanged || execution->waitFlagsChanged) + { + FreeWaitEventSet(execution->waitEventSet); + + execution->waitEventSet = BuildWaitEventSet(execution->sessionList); + + if (execution->connectionSetChanged) + { + /* + * The execution might take a while, so explicitly free at this point + * because we don't need anymore. + */ + pfree(events); + + /* recalculate (and allocate) since the sessions have changed */ + eventSetSize = list_length(execution->sessionList) + 2; + + events = palloc0(eventSetSize * sizeof(WaitEvent)); + } + + execution->connectionSetChanged = false; + execution->waitFlagsChanged = false; + } + + /* wait for I/O events */ +#if (PG_VERSION_NUM >= 100000) + eventCount = WaitEventSetWait(execution->waitEventSet, timeout, events, + eventSetSize, WAIT_EVENT_CLIENT_READ); +#else + eventCount = WaitEventSetWait(execution->waitEventSet, timeout, events, + eventSetSize); +#endif + + /* process I/O events */ + for (; eventIndex < eventCount; eventIndex++) + { + WaitEvent *event = &events[eventIndex]; + WorkerSession *session = NULL; + + if (event->events & WL_POSTMASTER_DEATH) + { + ereport(ERROR, (errmsg("postmaster was shut down, exiting"))); + } + + if (event->events & WL_LATCH_SET) + { + ResetLatch(MyLatch); + + if (execution->raiseInterrupts) + { + CHECK_FOR_INTERRUPTS(); + } + + if (InterruptHoldoffCount > 0 && (QueryCancelPending || + ProcDiePending)) + { + /* + * Break out of event loop immediately in case of cancellation. + * We cannot use "return" here inside a PG_TRY() block since + * then the exception stack won't be reset. + */ + cancellationReceived = true; + break; + } + + continue; + } + + session = (WorkerSession *) event->user_data; + + ConnectionStateMachine(session); + } + } + + pfree(events); + FreeWaitEventSet(execution->waitEventSet); + + CleanUpSessions(execution); + } + PG_CATCH(); + { + /* + * We can still recover from error using ROLLBACK TO SAVEPOINT, + * unclaim all connections to allow that. + */ + UnclaimAllSessionConnections(execution->sessionList); + + FreeWaitEventSet(execution->waitEventSet); + + PG_RE_THROW(); + } + PG_END_TRY(); +} + + +/* + * ManageWorkerPool ensures the worker pool has the appropriate number of connections + * based on the number of pending tasks. + */ +static void +ManageWorkerPool(WorkerPool *workerPool) +{ + DistributedExecution *execution = workerPool->distributedExecution; + WorkerNode *workerNode = workerPool->node; + int targetPoolSize = execution->targetPoolSize; + int initiatedConnectionCount = list_length(workerPool->sessionList); + int activeConnectionCount PG_USED_FOR_ASSERTS_ONLY = + workerPool->activeConnectionCount; + int idleConnectionCount PG_USED_FOR_ASSERTS_ONLY = + workerPool->idleConnectionCount; + int failedConnectionCount = workerPool->failedConnectionCount; + int readyTaskCount = workerPool->readyTaskCount; + int newConnectionCount = 0; + int connectionIndex = 0; + + /* we should always have more (or equal) active connections than idle connections */ + Assert(activeConnectionCount >= idleConnectionCount); + + /* we should always have more (or equal) initiated connections than active connections */ + Assert(initiatedConnectionCount >= activeConnectionCount); + + /* we should never have less than 0 connections ever */ + Assert(activeConnectionCount >= 0 && idleConnectionCount >= 0); + + if (workerPool->failed) + { + /* connection pool failed */ + return; + } + + /* we might fail the execution or warn the user about connection timeouts */ + CheckConnectionTimeout(workerPool); + + if (failedConnectionCount >= 1) + { + /* do not attempt to open more connections after one failed */ + return; + } + + if (UseConnectionPerPlacement()) + { + int unusedConnectionCount = workerPool->unusedConnectionCount; + + /* + * If force_max_query_parallelization is enabled then we ignore pool size + * and idle connections. Instead, we open new connections as long as there + * are more tasks than unused connections. + */ + + newConnectionCount = Max(readyTaskCount - unusedConnectionCount, 0); + } + else + { + /* cannot open more than targetPoolSize connections */ + int maxNewConnectionCount = targetPoolSize - initiatedConnectionCount; + + /* total number of connections that are (almost) available for tasks */ + int usableConnectionCount = UsableConnectionCount(workerPool); + + /* + * Number of additional connections we would need to run all ready tasks in + * parallel. + */ + int newConnectionsForReadyTasks = readyTaskCount - usableConnectionCount; + + /* + * Open enough connections to handle all tasks that are ready, but no more + * than the target pool size. + */ + newConnectionCount = Min(newConnectionsForReadyTasks, maxNewConnectionCount); + + if (newConnectionCount > 0 && ExecutorSlowStartInterval > 0) + { + TimestampTz now = GetCurrentTimestamp(); + + if (TimestampDifferenceExceeds(workerPool->lastConnectionOpenTime, now, + ExecutorSlowStartInterval)) + { + newConnectionCount = Min(newConnectionCount, + workerPool->maxNewConnectionsPerCycle); + + /* increase the open rate every cycle (like TCP slow start) */ + workerPool->maxNewConnectionsPerCycle += 1; + } + else + { + /* wait a bit until opening more connections */ + return; + } + } + } + + if (newConnectionCount <= 0) + { + return; + } + + elog(DEBUG4, "opening %d new connections to %s:%d", newConnectionCount, + workerNode->workerName, workerNode->workerPort); + + for (connectionIndex = 0; connectionIndex < newConnectionCount; connectionIndex++) + { + MultiConnection *connection = NULL; + WorkerSession *session = NULL; + + /* experimental: just to see the perf benefits of caching connections */ + int connectionFlags = 0; + + /* open a new connection to the worker */ + connection = StartNodeUserDatabaseConnection(connectionFlags, + workerNode->workerName, + workerNode->workerPort, + NULL, NULL); + + /* + * Assign the initial state in the connection state machine. The connection + * may already be open, but ConnectionStateMachine will immediately detect + * this. + */ + connection->connectionState = MULTI_CONNECTION_CONNECTING; + + /* + * Ensure that subsequent calls to StartNodeUserDatabaseConnection get a + * different connection. + */ + connection->claimedExclusively = true; + + /* create a session for the connection */ + session = FindOrCreateWorkerSession(workerPool, connection); + + /* always poll the connection in the first round */ + UpdateConnectionWaitFlags(session, WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + } + + workerPool->lastConnectionOpenTime = GetCurrentTimestamp(); + execution->connectionSetChanged = true; +} + + +/* + * CheckConnectionTimeout makes sure that the execution enforces the connection + * establishment timeout defined by the user (NodeConnectionTimeout). + * + * The rule is that if a worker pool has already initiated connection establishment + * and has not succeeded to finish establishments that are necessary to execute tasks, + * take an action. For the types of actions, see the comments in the function. + * + * Enforcing the timeout per pool (over per session) helps the execution to continue + * even if we can establish a single connection as we expect to have target pool size + * number of connections. In the end, the executor is capable of using one connection + * to execute multiple tasks. + */ +static void +CheckConnectionTimeout(WorkerPool *workerPool) +{ + DistributedExecution *execution = workerPool->distributedExecution; + TimestampTz poolStartTime = workerPool->poolStartTime; + TimestampTz now = GetCurrentTimestamp(); + + int initiatedConnectionCount = list_length(workerPool->sessionList); + int activeConnectionCount = workerPool->activeConnectionCount; + int requiredActiveConnectionCount = 1; + + if (initiatedConnectionCount == 0) + { + /* no connection has been planned for the pool yet */ + Assert(poolStartTime == 0); + return; + } + + /* + * This is a special case where we assign tasks to sessions even before + * the connections are established. So, make sure to apply similar + * restrictions. In this case, make sure that we get all the connections + * established. + */ + if (UseConnectionPerPlacement()) + { + requiredActiveConnectionCount = initiatedConnectionCount; + } + + if (TimestampDifferenceExceeds(poolStartTime, now, NodeConnectionTimeout)) + { + if (activeConnectionCount < requiredActiveConnectionCount) + { + int logLevel = WARNING; + + /* + * First fail the pool and create an opportunity to execute tasks + * over other pools when tasks have more than one placement to execute. + */ + WorkerPoolFailed(workerPool); + + /* + * The enforcement is not always erroring out. For example, if a SELECT task + * has two different placements, we'd warn the user, fail the pool and continue + * with the next placement. + */ + if (execution->errorOnAnyFailure || execution->failed) + { + logLevel = ERROR; + } + + ereport(logLevel, (errcode(ERRCODE_CONNECTION_FAILURE), + errmsg("could not establish any connections to the node " + "%s:%d after %u ms", workerPool->node->workerName, + workerPool->node->workerPort, + NodeConnectionTimeout))); + } + else + { + /* stop interrupting WaitEventSetWait for timeouts */ + workerPool->checkForPoolTimeout = false; + } + } +} + + +/* + * UsableConnectionCount returns the number of connections in the worker pool + * that are (soon to be) usable for sending commands, this includes both idle + * connections and connections that are still establishing. + */ +static int +UsableConnectionCount(WorkerPool *workerPool) +{ + int initiatedConnectionCount = list_length(workerPool->sessionList); + int activeConnectionCount = workerPool->activeConnectionCount; + int failedConnectionCount = workerPool->failedConnectionCount; + int idleConnectionCount = workerPool->idleConnectionCount; + + /* connections that are still establishing will soon be available for tasks */ + int establishingConnectionCount = + initiatedConnectionCount - activeConnectionCount - failedConnectionCount; + + int usableConnectionCount = idleConnectionCount + establishingConnectionCount; + + return usableConnectionCount; +} + + +/* + * NextEventTimeout finds the earliest time at which we need to interrupt + * WaitEventSetWait because of a timeout and returns the number of milliseconds + * until that event with a minimum of 1ms and a maximum of 1000ms. + * + * This code may be sensitive to clock jumps, but only has the effect of waking + * up WaitEventSetWait slightly earlier to later. + */ +static long +NextEventTimeout(DistributedExecution *execution) +{ + ListCell *workerCell = NULL; + TimestampTz now = GetCurrentTimestamp(); + long eventTimeout = 1000; /* milliseconds */ + + foreach(workerCell, execution->workerList) + { + WorkerPool *workerPool = (WorkerPool *) lfirst(workerCell); + int initiatedConnectionCount = 0; + + if (workerPool->failed) + { + /* worker pool may have already timed out */ + continue; + } + + if (workerPool->poolStartTime != 0 && workerPool->checkForPoolTimeout) + { + long timeSincePoolStartMs = + MillisecondsBetweenTimestamps(workerPool->poolStartTime, now); + + /* + * This could go into the negative if the connection timeout just passed. + * In that case we want to wake up as soon as possible. Once the timeout + * has been processed, checkForPoolTimeout will be false so we will skip + * this check. + */ + long timeUntilConnectionTimeoutMs = + NodeConnectionTimeout - timeSincePoolStartMs; + + if (timeUntilConnectionTimeoutMs < eventTimeout) + { + eventTimeout = timeUntilConnectionTimeoutMs; + } + } + + initiatedConnectionCount = list_length(workerPool->sessionList); + + /* + * If there are connections to open we wait at most up to the end of the + * current slow start interval. + */ + if (workerPool->readyTaskCount > UsableConnectionCount(workerPool) && + initiatedConnectionCount < execution->targetPoolSize) + { + long timeSinceLastConnectMs = + MillisecondsBetweenTimestamps(workerPool->lastConnectionOpenTime, now); + long timeUntilSlowStartInterval = + ExecutorSlowStartInterval - timeSinceLastConnectMs; + + if (timeUntilSlowStartInterval < eventTimeout) + { + eventTimeout = timeUntilSlowStartInterval; + } + } + } + + return Max(1, eventTimeout); +} + + +/* + * MillisecondsBetweenTimestamps is a helper to get the number of milliseconds + * between timestamps when it is expected to be small enough to fit in a + * long. + */ +static long +MillisecondsBetweenTimestamps(TimestampTz startTime, TimestampTz endTime) +{ + long secs = 0; + int micros = 0; + + TimestampDifference(startTime, endTime, &secs, µs); + + return secs * 1000 + micros / 1000; +} + + +/* + * ConnectionStateMachine opens a connection and descends into the transaction + * state machine when ready. + */ +static void +ConnectionStateMachine(WorkerSession *session) +{ + WorkerPool *workerPool = session->workerPool; + DistributedExecution *execution = workerPool->distributedExecution; + + MultiConnection *connection = session->connection; + MultiConnectionState currentState; + + do { + currentState = connection->connectionState; + + switch (currentState) + { + case MULTI_CONNECTION_INITIAL: + { + /* simply iterate the state machine */ + connection->connectionState = MULTI_CONNECTION_CONNECTING; + break; + } + + case MULTI_CONNECTION_CONNECTING: + { + PostgresPollingStatusType pollMode; + + ConnStatusType status = PQstatus(connection->pgConn); + if (status == CONNECTION_OK) + { + elog(DEBUG4, "established connection to %s:%d for session %ld", + connection->hostname, connection->port, session->sessionId); + + workerPool->activeConnectionCount++; + workerPool->idleConnectionCount++; + + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + + connection->connectionState = MULTI_CONNECTION_CONNECTED; + break; + } + else if (status == CONNECTION_BAD) + { + connection->connectionState = MULTI_CONNECTION_FAILED; + break; + } + + pollMode = PQconnectPoll(connection->pgConn); + if (pollMode == PGRES_POLLING_FAILED) + { + connection->connectionState = MULTI_CONNECTION_FAILED; + } + else if (pollMode == PGRES_POLLING_READING) + { + UpdateConnectionWaitFlags(session, WL_SOCKET_READABLE); + } + else if (pollMode == PGRES_POLLING_WRITING) + { + UpdateConnectionWaitFlags(session, WL_SOCKET_WRITEABLE); + } + else + { + elog(DEBUG4, "established connection to %s:%d for session %ld", + connection->hostname, connection->port, session->sessionId); + + workerPool->activeConnectionCount++; + workerPool->idleConnectionCount++; + + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + + connection->connectionState = MULTI_CONNECTION_CONNECTED; + } + + break; + } + + case MULTI_CONNECTION_CONNECTED: + { + /* connection is ready, run the transaction state machine */ + TransactionStateMachine(session); + break; + } + + case MULTI_CONNECTION_LOST: + { + /* managed to connect, but connection was lost */ + workerPool->activeConnectionCount--; + + if (session->currentTask == NULL) + { + /* this was an idle connection */ + workerPool->idleConnectionCount--; + } + + connection->connectionState = MULTI_CONNECTION_FAILED; + break; + } + + case MULTI_CONNECTION_FAILED: + { + /* connection failed or was lost */ + int totalConnectionCount = list_length(workerPool->sessionList); + + workerPool->failedConnectionCount++; + + /* if the connection executed a critical command it should fail */ + MarkRemoteTransactionFailed(connection, false); + + /* mark all assigned placement executions as failed */ + WorkerSessionFailed(session); + + if (workerPool->failedConnectionCount >= totalConnectionCount) + { + /* + * All current connection attempts have failed. + * Mark all unassigned placement executions as failed. + * + * We do not currently retry if the first connection + * attempt fails. + */ + WorkerPoolFailed(workerPool); + } + + /* + * The execution may have failed as a result of WorkerSessionFailed + * or WorkerPoolFailed. + */ + if (execution->failed || execution->errorOnAnyFailure) + { + /* a task has failed due to this connection failure */ + ReportConnectionError(connection, ERROR); + } + else + { + /* can continue with the remaining nodes */ + ReportConnectionError(connection, WARNING); + } + + /* remove the connection */ + UnclaimConnection(connection); + + /* + * We forcefully close the underlying libpq connection because + * we don't want any subsequent execution (either subPlan executions + * or new command executions within a transaction block) use the + * connection. + * + * However, we prefer to keep the MultiConnection around until + * the end of FinishDistributedExecution() to simplify the code. + * Thus, we prefer ShutdownConnection() over CloseConnection(). + */ + ShutdownConnection(connection); + + /* remove connection from wait event set */ + execution->connectionSetChanged = true; + + /* + * Reset the transaction state machine since CloseConnection() + * relies on it and even if we're not inside a distributed transaction + * we set the transaction state (e.g., REMOTE_TRANS_SENT_COMMAND). + */ + if (!connection->remoteTransaction.beginSent) + { + connection->remoteTransaction.transactionState = + REMOTE_TRANS_INVALID; + } + + break; + } + + default: + { + break; + } + } + } while (connection->connectionState != currentState); +} + + +/* + * Activate2PCIfModifyingTransactionExpandsToNewNode sets the coordinated + * transaction to use 2PC under the following circumstances: + * - We're already in a transaction block + * - At least one of the previous commands in the transaction block + * made a modification, which have not set 2PC itself because it + * was a single shard command + * - The input "session" is used for a distributed execution which + * modifies the database. However, the session (and hence the + * connection) is established to a different worker than the ones + * that is used previously in the transaction. + * + * To give an example, + * BEGIN; + * -- assume that the following INSERT goes to worker-A + * -- also note that this single command does not activate + * -- 2PC itself since it is a single shard mofication + * INSERT INTO distributed_table (dist_key) VALUES (1); + * + * -- do one more single shard UPDATE hitting the same + * shard (or worker node in general) + * -- this wouldn't activate 2PC, since we're operating on the + * -- same worker node that we've modified earlier + * -- so the executor would use the same connection + * UPDATE distributed_table SET value = 10 WHERE dist_key = 1; + * + * -- now, do one more INSERT, which goes to worker-B + * -- At this point, this function would activate 2PC + * -- since we're now expanding to a new node + * -- for example, if this command were a SELECT, we wouldn't + * -- activate 2PC since we're only interested in modifications/DDLs + * INSERT INTO distributed_table (dist_key) VALUES (2); + */ +static void +Activate2PCIfModifyingTransactionExpandsToNewNode(WorkerSession *session) +{ + DistributedExecution *execution = NULL; + + if (MultiShardCommitProtocol != COMMIT_PROTOCOL_2PC) + { + /* we don't need 2PC, so no need to continue */ + return; + } + + execution = session->workerPool->distributedExecution; + if (TransactionModifiedDistributedTable(execution) && + DistributedExecutionModifiesDatabase(execution) && + !ConnectionModifiedPlacement(session->connection)) + { + /* + * We already did a modification, but not on the connection that we + * just opened, which means we're now going to make modifications + * over multiple connections. Activate 2PC! + */ + CoordinatedTransactionUse2PC(); + } +} + + +/* + * TransactionModifiedDistributedTable returns true if the current transaction already + * executed a command which modified at least one distributed table in the current + * transaction. + */ +static bool +TransactionModifiedDistributedTable(DistributedExecution *execution) +{ + /* + * We need to explicitly check for isTransaction due to + * citus.function_opens_transaction_block flag. When set to false, we + * should not be pretending that we're in a coordinated transaction even + * if XACT_MODIFICATION_DATA is set. That's why we implemented this workaround. + */ + return execution->isTransaction && XactModificationLevel == XACT_MODIFICATION_DATA; +} + + +/* + * TransactionStateMachine manages the execution of tasks over a connection. + */ +static void +TransactionStateMachine(WorkerSession *session) +{ + WorkerPool *workerPool = session->workerPool; + DistributedExecution *execution = workerPool->distributedExecution; + + MultiConnection *connection = session->connection; + RemoteTransaction *transaction = &(connection->remoteTransaction); + RemoteTransactionState currentState; + + do { + currentState = transaction->transactionState; + + if (!CheckConnectionReady(session)) + { + /* connection is busy, no state transitions to make */ + break; + } + + switch (currentState) + { + case REMOTE_TRANS_INVALID: + { + if (execution->isTransaction) + { + /* if we're expanding the nodes in a transaction, use 2PC */ + Activate2PCIfModifyingTransactionExpandsToNewNode(session); + + /* need to open a transaction block first */ + StartRemoteTransactionBegin(connection); + + transaction->transactionState = REMOTE_TRANS_CLEARING_RESULTS; + } + else + { + TaskPlacementExecution *placementExecution = NULL; + + placementExecution = PopPlacementExecution(session); + if (placementExecution == NULL) + { + /* + * No tasks are ready to be executed at the moment. But we + * still mark the socket readable to get any notices if exists. + */ + UpdateConnectionWaitFlags(session, WL_SOCKET_READABLE); + + break; + } + + StartPlacementExecutionOnSession(placementExecution, session); + transaction->transactionState = REMOTE_TRANS_SENT_COMMAND; + } + + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + break; + } + + case REMOTE_TRANS_SENT_BEGIN: + case REMOTE_TRANS_CLEARING_RESULTS: + { + PGresult *result = NULL; + + result = PQgetResult(connection->pgConn); + if (result != NULL) + { + if (!IsResponseOK(result)) + { + /* query failures are always hard errors */ + ReportResultError(connection, result, ERROR); + } + + PQclear(result); + + /* wake up WaitEventSetWait */ + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + + break; + } + + if (session->currentTask != NULL) + { + TaskPlacementExecution *placementExecution = session->currentTask; + bool succeeded = true; + + /* + * Once we finished a task on a connection, we no longer + * allow that connection to fail. + */ + MarkRemoteTransactionCritical(connection); + + session->currentTask = NULL; + + PlacementExecutionDone(placementExecution, succeeded); + + /* connection is ready to use for executing commands */ + workerPool->idleConnectionCount++; + } + + /* connection needs to be writeable to send next command */ + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + + if (execution->isTransaction) + { + transaction->transactionState = REMOTE_TRANS_STARTED; + } + else + { + transaction->transactionState = REMOTE_TRANS_INVALID; + } + break; + } + + case REMOTE_TRANS_STARTED: + { + TaskPlacementExecution *placementExecution = NULL; + + placementExecution = PopPlacementExecution(session); + if (placementExecution == NULL) + { + /* no tasks are ready to be executed at the moment */ + UpdateConnectionWaitFlags(session, WL_SOCKET_READABLE); + break; + } + + StartPlacementExecutionOnSession(placementExecution, session); + transaction->transactionState = REMOTE_TRANS_SENT_COMMAND; + break; + } + + case REMOTE_TRANS_SENT_COMMAND: + { + bool fetchDone = false; + TaskPlacementExecution *placementExecution = session->currentTask; + ShardCommandExecution *shardCommandExecution = + placementExecution->shardCommandExecution; + bool storeRows = shardCommandExecution->expectResults; + + if (shardCommandExecution->gotResults) + { + /* already received results from another replica */ + storeRows = false; + } + + fetchDone = ReceiveResults(session, storeRows); + if (!fetchDone) + { + break; + } + + shardCommandExecution->gotResults = true; + transaction->transactionState = REMOTE_TRANS_CLEARING_RESULTS; + break; + } + + default: + { + break; + } + } + } + + /* iterate in case we can perform multiple transitions at once */ + while (transaction->transactionState != currentState); +} + + +/* + * UpdateConnectionWaitFlags is a wrapper around setting waitFlags of the connection. + * + * This function might further improved in a sense that to use use ModifyWaitEvent on + * waitFlag changes as opposed to what we do now: always rebuild the wait event sets. + * Our initial benchmarks didn't show any significant performance improvements, but + * good to keep in mind the potential improvements. + */ +static void +UpdateConnectionWaitFlags(WorkerSession *session, int waitFlags) +{ + MultiConnection *connection = session->connection; + DistributedExecution *execution = session->workerPool->distributedExecution; + + /* do not take any actions if the flags not changed */ + if (connection->waitFlags == waitFlags) + { + return; + } + + connection->waitFlags = waitFlags; + + /* without signalling the execution, the flag changes won't be reflected */ + execution->waitFlagsChanged = true; +} + + +/* + * CheckConnectionReady returns true if the the connection is ready to + * read or write, or false if it still has bytes to send/receive. + */ +static bool +CheckConnectionReady(WorkerSession *session) +{ + int sendStatus = 0; + MultiConnection *connection = session->connection; + int waitFlags = WL_SOCKET_READABLE; + bool connectionReady = false; + + ConnStatusType status = PQstatus(connection->pgConn); + if (status == CONNECTION_BAD) + { + connection->connectionState = MULTI_CONNECTION_LOST; + return false; + } + + /* try to send all pending data */ + sendStatus = PQflush(connection->pgConn); + if (sendStatus == -1) + { + connection->connectionState = MULTI_CONNECTION_LOST; + return false; + } + else if (sendStatus == 1) + { + /* more data to send, wait for socket to become writable */ + waitFlags = waitFlags | WL_SOCKET_WRITEABLE; + } + + /* if reading fails, there's not much we can do */ + if (PQconsumeInput(connection->pgConn) == 0) + { + connection->connectionState = MULTI_CONNECTION_LOST; + return false; + } + + if (!PQisBusy(connection->pgConn)) + { + connectionReady = true; + } + + UpdateConnectionWaitFlags(session, waitFlags); + + return connectionReady; +} + + +/* + * PopPlacementExecution returns the next available assigned or unassigned + * placement execution for the given session. + */ +static TaskPlacementExecution * +PopPlacementExecution(WorkerSession *session) +{ + TaskPlacementExecution *placementExecution = NULL; + WorkerPool *workerPool = session->workerPool; + + placementExecution = PopAssignedPlacementExecution(session); + if (placementExecution == NULL) + { + if (session->commandsSent > 0 && UseConnectionPerPlacement()) + { + /* + * Only send one command per connection if force_max_query_parallelisation + * is enabled, unless it's an assigned placement execution. + */ + return NULL; + } + + /* no more assigned tasks, pick an unassigned task */ + placementExecution = PopUnassignedPlacementExecution(workerPool); + } + + return placementExecution; +} + + +/* + * PopAssignedPlacementExecution finds an executable task from the queue of assigned tasks. + */ +static TaskPlacementExecution * +PopAssignedPlacementExecution(WorkerSession *session) +{ + TaskPlacementExecution *placementExecution = NULL; + dlist_head *readyTaskQueue = &(session->readyTaskQueue); + + if (dlist_is_empty(readyTaskQueue)) + { + return NULL; + } + + placementExecution = dlist_container(TaskPlacementExecution, + sessionReadyQueueNode, + dlist_pop_head_node(readyTaskQueue)); + + return placementExecution; +} + + +/* + * PopAssignedPlacementExecution finds an executable task from the queue of assigned tasks. + */ +static TaskPlacementExecution * +PopUnassignedPlacementExecution(WorkerPool *workerPool) +{ + TaskPlacementExecution *placementExecution = NULL; + dlist_head *readyTaskQueue = &(workerPool->readyTaskQueue); + + if (dlist_is_empty(readyTaskQueue)) + { + return NULL; + } + + placementExecution = dlist_container(TaskPlacementExecution, + workerReadyQueueNode, + dlist_pop_head_node(readyTaskQueue)); + + workerPool->readyTaskCount--; + + return placementExecution; +} + + +/* + * StartPlacementExecutionOnSession gets a TaskPlacementExecition and + * WorkerSession, the task's query is sent to the worker via the session. + * + * The function does some bookkeeping such as associating the placement + * accesses with the connection and updating session's local variables. For + * details read the comments in the function. + * + * The function returns true if the query is successfully sent over the + * connection, otherwise false. + */ +static bool +StartPlacementExecutionOnSession(TaskPlacementExecution *placementExecution, + WorkerSession *session) +{ + WorkerPool *workerPool = session->workerPool; + DistributedExecution *execution = workerPool->distributedExecution; + ParamListInfo paramListInfo = execution->paramListInfo; + MultiConnection *connection = session->connection; + ShardCommandExecution *shardCommandExecution = + placementExecution->shardCommandExecution; + Task *task = shardCommandExecution->task; + ShardPlacement *taskPlacement = placementExecution->shardPlacement; + List *placementAccessList = PlacementAccessListForTask(task, taskPlacement); + char *queryString = task->queryString; + int querySent = 0; + int singleRowMode = 0; + + /* + * Make sure that subsequent commands on the same placement + * use the same connection. + */ + AssignPlacementListToConnection(placementAccessList, connection); + + /* one more command is sent over the session */ + session->commandsSent++; + + if (session->commandsSent == 1) + { + /* first time we send a command, consider the connection used (not unused) */ + workerPool->unusedConnectionCount--; + } + + /* connection is going to be in use */ + workerPool->idleConnectionCount--; + + if (paramListInfo != NULL) + { + int parameterCount = paramListInfo->numParams; + Oid *parameterTypes = NULL; + const char **parameterValues = NULL; + + /* force evaluation of bound params */ + paramListInfo = copyParamList(paramListInfo); + + ExtractParametersFromParamListInfo(paramListInfo, ¶meterTypes, + ¶meterValues); + querySent = SendRemoteCommandParams(connection, queryString, parameterCount, + parameterTypes, parameterValues); + } + else + { + querySent = SendRemoteCommand(connection, queryString); + } + + if (querySent == 0) + { + connection->connectionState = MULTI_CONNECTION_LOST; + return false; + } + + singleRowMode = PQsetSingleRowMode(connection->pgConn); + if (singleRowMode == 0) + { + connection->connectionState = MULTI_CONNECTION_LOST; + return false; + } + + session->currentTask = placementExecution; + placementExecution->executionState = PLACEMENT_EXECUTION_RUNNING; + + return true; +} + + +/* + * PlacementAccessListForTask returns a list of placement accesses for a given + * task and task placement. + */ +static List * +PlacementAccessListForTask(Task *task, ShardPlacement *taskPlacement) +{ + List *placementAccessList = NIL; + List *relationShardList = task->relationShardList; + bool addAnchorAccess = false; + ShardPlacementAccessType accessType = PLACEMENT_ACCESS_SELECT; + + if (task->taskType == MODIFY_TASK) + { + /* DML command */ + addAnchorAccess = true; + accessType = PLACEMENT_ACCESS_DML; + } + else if (task->taskType == DDL_TASK || task->taskType == VACUUM_ANALYZE_TASK) + { + /* DDL command */ + addAnchorAccess = true; + accessType = PLACEMENT_ACCESS_DDL; + } + else if (relationShardList == NIL) + { + /* SELECT query that does not touch any shard placements */ + addAnchorAccess = true; + accessType = PLACEMENT_ACCESS_SELECT; + } + + if (addAnchorAccess) + { + ShardPlacementAccess *placementAccess = + CreatePlacementAccess(taskPlacement, accessType); + + placementAccessList = lappend(placementAccessList, placementAccess); + } + + /* + * We've already added anchor shardId's placement access to the list. Now, + * add the other placements in the relationShardList. + */ + if (accessType == PLACEMENT_ACCESS_DDL) + { + /* + * All relations appearing inter-shard DDL commands should be marked + * with DDL access. + */ + List *relationShardAccessList = + BuildPlacementDDLList(taskPlacement->groupId, relationShardList); + + placementAccessList = list_concat(placementAccessList, relationShardAccessList); + } + else + { + /* + * In case of SELECTs or DML's, we add SELECT placement accesses to the + * elements in relationShardList. For SELECT queries, it is trivial, since + * the query is literally accesses the relationShardList in the same query. + * + * For DMLs, create placement accesses for placements that appear in a + * subselect. + */ + List *relationShardAccessList = + BuildPlacementSelectList(taskPlacement->groupId, relationShardList); + + placementAccessList = list_concat(placementAccessList, relationShardAccessList); + } + + return placementAccessList; +} + + +/* + * ReceiveResults reads the result of a command or query and writes returned + * rows to the tuple store of the scan state. It returns whether fetching results + * were done. On failure, it throws an error. + */ +static bool +ReceiveResults(WorkerSession *session, bool storeRows) +{ + bool fetchDone = false; + MultiConnection *connection = session->connection; + WorkerPool *workerPool = session->workerPool; + DistributedExecution *execution = workerPool->distributedExecution; + DistributedExecutionStats *executionStats = execution->executionStats; + TupleDesc tupleDescriptor = execution->tupleDescriptor; + AttInMetadata *attributeInputMetadata = NULL; + uint32 expectedColumnCount = 0; + char **columnArray = NULL; + Tuplestorestate *tupleStore = execution->tupleStore; + + MemoryContext ioContext = AllocSetContextCreate(CurrentMemoryContext, + "ReceiveResults", + ALLOCSET_DEFAULT_MINSIZE, + ALLOCSET_DEFAULT_INITSIZE, + ALLOCSET_DEFAULT_MAXSIZE); + if (tupleDescriptor != NULL) + { + attributeInputMetadata = TupleDescGetAttInMetadata(tupleDescriptor); + expectedColumnCount = tupleDescriptor->natts; + columnArray = (char **) palloc0(expectedColumnCount * sizeof(char *)); + } + + + while (!PQisBusy(connection->pgConn)) + { + uint32 rowIndex = 0; + uint32 columnIndex = 0; + uint32 rowsProcessed = 0; + uint32 columnCount = 0; + ExecStatusType resultStatus = 0; + + PGresult *result = PQgetResult(connection->pgConn); + if (result == NULL) + { + /* no more results, break out of loop and free allocated memory */ + fetchDone = true; + break; + } + + resultStatus = PQresultStatus(result); + if (resultStatus == PGRES_COMMAND_OK) + { + char *currentAffectedTupleString = PQcmdTuples(result); + int64 currentAffectedTupleCount = 0; + ShardCommandExecution *shardCommandExecution = + session->currentTask->shardCommandExecution; + + /* if there are multiple replicas, make sure to consider only one */ + if (!shardCommandExecution->gotResults && *currentAffectedTupleString != '\0') + { + scanint8(currentAffectedTupleString, false, ¤tAffectedTupleCount); + Assert(currentAffectedTupleCount >= 0); + + execution->rowsProcessed += currentAffectedTupleCount; + } + + PQclear(result); + + /* no more results, break out of loop and free allocated memory */ + fetchDone = true; + break; + } + else if (resultStatus == PGRES_TUPLES_OK) + { + /* + * We've already consumed all the tuples, no more results. Break out + * of loop and free allocated memory before returning. + */ + Assert(PQntuples(result) == 0); + PQclear(result); + + fetchDone = true; + break; + } + else if (resultStatus != PGRES_SINGLE_TUPLE) + { + /* query failures are always hard errors */ + ReportResultError(connection, result, ERROR); + } + else if (!storeRows) + { + /* + * Already receieved rows from executing on another shard placement or + * doesn't need at all (e.g., DDL). + */ + PQclear(result); + continue; + } + + rowsProcessed = PQntuples(result); + columnCount = PQnfields(result); + + if (columnCount != expectedColumnCount) + { + ereport(ERROR, (errmsg("unexpected number of columns from worker: %d, " + "expected %d", + columnCount, expectedColumnCount))); + } + + for (rowIndex = 0; rowIndex < rowsProcessed; rowIndex++) + { + HeapTuple heapTuple = NULL; + MemoryContext oldContext = NULL; + memset(columnArray, 0, columnCount * sizeof(char *)); + + for (columnIndex = 0; columnIndex < columnCount; columnIndex++) + { + if (PQgetisnull(result, rowIndex, columnIndex)) + { + columnArray[columnIndex] = NULL; + } + else + { + columnArray[columnIndex] = PQgetvalue(result, rowIndex, columnIndex); + if (SubPlanLevel > 0 && executionStats != NULL) + { + executionStats->totalIntermediateResultSize += PQgetlength(result, + rowIndex, + columnIndex); + } + } + } + + /* + * Switch to a temporary memory context that we reset after each tuple. This + * protects us from any memory leaks that might be present in I/O functions + * called by BuildTupleFromCStrings. + */ + oldContext = MemoryContextSwitchTo(ioContext); + + heapTuple = BuildTupleFromCStrings(attributeInputMetadata, columnArray); + + MemoryContextSwitchTo(oldContext); + + tuplestore_puttuple(tupleStore, heapTuple); + MemoryContextReset(ioContext); + + execution->rowsProcessed++; + } + + PQclear(result); + + if (executionStats != NULL && CheckIfSizeLimitIsExceeded(executionStats)) + { + ErrorSizeLimitIsExceeded(); + } + } + + if (columnArray != NULL) + { + pfree(columnArray); + } + + MemoryContextDelete(ioContext); + + return fetchDone; +} + + +/* + * WorkerPoolFailed marks a worker pool and all the placement executions scheduled + * on it as failed. + */ +static void +WorkerPoolFailed(WorkerPool *workerPool) +{ + bool succeeded = false; + dlist_iter iter; + ListCell *sessionCell = NULL; + + /* a pool cannot fail multiple times */ + Assert(!workerPool->failed); + + dlist_foreach(iter, &workerPool->pendingTaskQueue) + { + TaskPlacementExecution *placementExecution = + dlist_container(TaskPlacementExecution, workerPendingQueueNode, iter.cur); + + PlacementExecutionDone(placementExecution, succeeded); + } + + dlist_foreach(iter, &workerPool->readyTaskQueue) + { + TaskPlacementExecution *placementExecution = + dlist_container(TaskPlacementExecution, workerReadyQueueNode, iter.cur); + + PlacementExecutionDone(placementExecution, succeeded); + } + + foreach(sessionCell, workerPool->sessionList) + { + WorkerSession *session = lfirst(sessionCell); + + WorkerSessionFailed(session); + } + + /* we do not want more connections in this pool */ + workerPool->readyTaskCount = 0; + workerPool->failed = true; + + /* + * The reason is that when replication factor is > 1 and we are performing + * a SELECT, then we only establish connections for the specific placements + * that we will read from. However, when a worker pool fails, we will need + * to establish multiple new connection to other workers and the query + * can only succeed if all those connections are established. + */ + if (UseConnectionPerPlacement()) + { + ListCell *workerCell = NULL; + List *workerList = workerPool->distributedExecution->workerList; + + foreach(workerCell, workerList) + { + WorkerPool *pool = (WorkerPool *) lfirst(workerCell); + + /* failed pools or pools without any connection attempts ignored */ + if (pool->failed || pool->poolStartTime == 0) + { + continue; + } + + /* + * This should give another NodeConnectionTimeout until all + * the necessary connections are established. + */ + pool->poolStartTime = GetCurrentTimestamp(); + pool->checkForPoolTimeout = true; + } + } +} + + +/* + * WorkerSessionFailed marks all placement executions scheduled on the + * connection as failed. + */ +static void +WorkerSessionFailed(WorkerSession *session) +{ + TaskPlacementExecution *placementExecution = session->currentTask; + bool succeeded = false; + dlist_iter iter; + + if (placementExecution != NULL) + { + /* connection failed while a task was active */ + PlacementExecutionDone(placementExecution, succeeded); + } + + dlist_foreach(iter, &session->pendingTaskQueue) + { + placementExecution = + dlist_container(TaskPlacementExecution, sessionPendingQueueNode, iter.cur); + + PlacementExecutionDone(placementExecution, succeeded); + } + + dlist_foreach(iter, &session->readyTaskQueue) + { + placementExecution = + dlist_container(TaskPlacementExecution, sessionReadyQueueNode, iter.cur); + + PlacementExecutionDone(placementExecution, succeeded); + } +} + + +/* + * PlacementExecutionDone marks the given placement execution as done when + * the results have been received or a failure occurred and sets the succeeded + * flag accordingly. It also adds other placement executions of the same + * task to the appropriate ready queues. + */ +static void +PlacementExecutionDone(TaskPlacementExecution *placementExecution, bool succeeded) +{ + WorkerPool *workerPool = placementExecution->workerPool; + DistributedExecution *execution = workerPool->distributedExecution; + ShardCommandExecution *shardCommandExecution = + placementExecution->shardCommandExecution; + TaskExecutionState executionState = shardCommandExecution->executionState; + TaskExecutionState newExecutionState = TASK_EXECUTION_NOT_FINISHED; + bool failedPlacementExecutionIsOnPendingQueue = false; + + /* mark the placement execution as finished */ + if (succeeded) + { + placementExecution->executionState = PLACEMENT_EXECUTION_FINISHED; + } + else + { + if (ShouldMarkPlacementsInvalidOnFailure(execution)) + { + ShardPlacement *shardPlacement = placementExecution->shardPlacement; + + /* + * We only set shard state if its current state is FILE_FINALIZED, which + * prevents overwriting shard state if it is already set at somewhere else. + */ + if (shardPlacement->shardState == FILE_FINALIZED) + { + UpdateShardPlacementState(shardPlacement->placementId, FILE_INACTIVE); + } + } + + if (placementExecution->executionState == PLACEMENT_EXECUTION_NOT_READY) + { + /* + * If the placement is in NOT_READY state, it means that the placement + * execution is assigned to the pending queue of a failed pool or + * session. So, we should not schedule the next placement execution based + * on this failure. + */ + failedPlacementExecutionIsOnPendingQueue = true; + } + + placementExecution->executionState = PLACEMENT_EXECUTION_FAILED; + } + + if (executionState != TASK_EXECUTION_NOT_FINISHED) + { + /* + * Task execution has already been finished, no need to continue the + * next placement. + */ + return; + } + + /* + * Update unfinishedTaskCount only when state changes from not finished to + * finished or failed state. + */ + newExecutionState = TaskExecutionStateMachine(shardCommandExecution); + if (newExecutionState == TASK_EXECUTION_FINISHED) + { + execution->unfinishedTaskCount--; + return; + } + else if (newExecutionState == TASK_EXECUTION_FAILED) + { + execution->unfinishedTaskCount--; + + /* + * Even if a single task execution fails, there is no way to + * successfully finish the execution. + */ + execution->failed = true; + return; + } + else if (!failedPlacementExecutionIsOnPendingQueue) + { + ScheduleNextPlacementExecution(placementExecution, succeeded); + } +} + + +/* + * ScheduleNextPlacementExecution is triggered if the query needs to be + * executed on any or all placements in order and there is a placement on + * which the execution has not happened yet. If so make that placement + * ready-to-start by adding it to the appropriate queue. + */ +static void +ScheduleNextPlacementExecution(TaskPlacementExecution *placementExecution, bool succeeded) +{ + ShardCommandExecution *shardCommandExecution = + placementExecution->shardCommandExecution; + PlacementExecutionOrder executionOrder = shardCommandExecution->executionOrder; + + if ((executionOrder == EXECUTION_ORDER_ANY && !succeeded) || + executionOrder == EXECUTION_ORDER_SEQUENTIAL) + { + TaskPlacementExecution *nextPlacementExecution = NULL; + int placementExecutionCount PG_USED_FOR_ASSERTS_ONLY = + shardCommandExecution->placementExecutionCount; + + /* find a placement execution that is not yet marked as failed */ + do { + int nextPlacementExecutionIndex = + placementExecution->placementExecutionIndex + 1; + + /* if all tasks failed then we should already have errored out */ + Assert(nextPlacementExecutionIndex < placementExecutionCount); + + /* get the next placement in the planning order */ + nextPlacementExecution = + shardCommandExecution->placementExecutions[nextPlacementExecutionIndex]; + + if (nextPlacementExecution->executionState == PLACEMENT_EXECUTION_NOT_READY) + { + /* move the placement execution to the ready queue */ + PlacementExecutionReady(nextPlacementExecution); + } + } while (nextPlacementExecution->executionState == PLACEMENT_EXECUTION_FAILED); + } +} + + +/* + * ShouldMarkPlacementsInvalidOnFailure returns true if the failure + * should trigger marking placements invalid. + */ +static bool +ShouldMarkPlacementsInvalidOnFailure(DistributedExecution *execution) +{ + if (!DistributedExecutionModifiesDatabase(execution) || execution->errorOnAnyFailure) + { + /* + * Failures that do not modify the database (e.g., mainly SELECTs) should + * never lead to invalid placement. + * + * Failures that lead throwing error, no need to mark any placement + * invalid. + */ + return false; + } + + return true; +} + + +/* + * PlacementExecutionReady adds a placement execution to the ready queue when + * its dependent placement executions have finished. + */ +static void +PlacementExecutionReady(TaskPlacementExecution *placementExecution) +{ + WorkerPool *workerPool = placementExecution->workerPool; + + if (placementExecution->assignedSession != NULL) + { + WorkerSession *session = placementExecution->assignedSession; + MultiConnection *connection = session->connection; + RemoteTransaction *transaction = &(connection->remoteTransaction); + RemoteTransactionState transactionState = transaction->transactionState; + + if (placementExecution->executionState == PLACEMENT_EXECUTION_NOT_READY) + { + /* remove from not-ready task queue */ + dlist_delete(&placementExecution->sessionPendingQueueNode); + + /* add to ready-to-start task queue */ + dlist_push_tail(&session->readyTaskQueue, + &placementExecution->sessionReadyQueueNode); + } + + if (transactionState == REMOTE_TRANS_INVALID || + transactionState == REMOTE_TRANS_STARTED) + { + /* + * If the connection is idle, wake it up by checking whether + * the connection is writeable. + */ + UpdateConnectionWaitFlags(session, WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + } + } + else + { + ListCell *sessionCell = NULL; + + if (placementExecution->executionState == PLACEMENT_EXECUTION_NOT_READY) + { + /* remove from not-ready task queue */ + dlist_delete(&placementExecution->workerPendingQueueNode); + + /* add to ready-to-start task queue */ + dlist_push_tail(&workerPool->readyTaskQueue, + &placementExecution->workerReadyQueueNode); + } + + workerPool->readyTaskCount++; + + /* wake up an idle connection by checking whether the connection is writeable */ + foreach(sessionCell, workerPool->sessionList) + { + WorkerSession *session = lfirst(sessionCell); + MultiConnection *connection = session->connection; + RemoteTransaction *transaction = &(connection->remoteTransaction); + RemoteTransactionState transactionState = transaction->transactionState; + + if (transactionState == REMOTE_TRANS_INVALID || + transactionState == REMOTE_TRANS_STARTED) + { + UpdateConnectionWaitFlags(session, + WL_SOCKET_READABLE | WL_SOCKET_WRITEABLE); + + break; + } + } + } + + /* update the state to ready for further processing */ + placementExecution->executionState = PLACEMENT_EXECUTION_READY; +} + + +/* + * TaskExecutionStateMachine returns whether a shard command execution + * finished or failed according to its execution order. If the task is + * already finished, simply return the state. Else, calculate the state + * and return it. + */ +static TaskExecutionState +TaskExecutionStateMachine(ShardCommandExecution *shardCommandExecution) +{ + PlacementExecutionOrder executionOrder = shardCommandExecution->executionOrder; + int donePlacementCount = 0; + int failedPlacementCount = 0; + int placementCount = 0; + int placementExecutionIndex = 0; + int placementExecutionCount = shardCommandExecution->placementExecutionCount; + TaskExecutionState currentTaskExecutionState = shardCommandExecution->executionState; + + if (currentTaskExecutionState != TASK_EXECUTION_NOT_FINISHED) + { + /* we've already calculated the state, simply return it */ + return currentTaskExecutionState; + } + + for (; placementExecutionIndex < placementExecutionCount; placementExecutionIndex++) + { + TaskPlacementExecution *placementExecution = + shardCommandExecution->placementExecutions[placementExecutionIndex]; + TaskPlacementExecutionState executionState = placementExecution->executionState; + + if (executionState == PLACEMENT_EXECUTION_FINISHED) + { + donePlacementCount++; + } + else if (executionState == PLACEMENT_EXECUTION_FAILED) + { + failedPlacementCount++; + } + + placementCount++; + } + + if (failedPlacementCount == placementCount) + { + currentTaskExecutionState = TASK_EXECUTION_FAILED; + } + else if (executionOrder == EXECUTION_ORDER_ANY && donePlacementCount > 0) + { + currentTaskExecutionState = TASK_EXECUTION_FINISHED; + } + else if (donePlacementCount + failedPlacementCount == placementCount) + { + currentTaskExecutionState = TASK_EXECUTION_FINISHED; + } + else + { + currentTaskExecutionState = TASK_EXECUTION_NOT_FINISHED; + } + + shardCommandExecution->executionState = currentTaskExecutionState; + + return shardCommandExecution->executionState; +} + + +/* + * BuildWaitEventSet creates a WaitEventSet for the given array of connections + * which can be used to wait for any of the sockets to become read-ready or + * write-ready. + */ +static WaitEventSet * +BuildWaitEventSet(List *sessionList) +{ + WaitEventSet *waitEventSet = NULL; + ListCell *sessionCell = NULL; + + /* additional 2 is for postmaster and latch */ + int eventSetSize = list_length(sessionList) + 2; + + waitEventSet = + CreateWaitEventSet(CurrentMemoryContext, eventSetSize); + + foreach(sessionCell, sessionList) + { + WorkerSession *session = lfirst(sessionCell); + MultiConnection *connection = session->connection; + int socket = 0; + + if (connection->pgConn == NULL) + { + /* connection died earlier in the transaction */ + continue; + } + + if (connection->waitFlags == 0) + { + /* not currently waiting for this connection */ + continue; + } + + socket = PQsocket(connection->pgConn); + if (socket == -1) + { + /* connection was closed */ + continue; + } + + AddWaitEventToSet(waitEventSet, connection->waitFlags, socket, NULL, + (void *) session); + } + + AddWaitEventToSet(waitEventSet, WL_POSTMASTER_DEATH, PGINVALID_SOCKET, NULL, NULL); + AddWaitEventToSet(waitEventSet, WL_LATCH_SET, PGINVALID_SOCKET, MyLatch, NULL); + + return waitEventSet; +} + + +/* + * SetLocalForceMaxQueryParallelization simply a C interface for + * setting the following: + * SET LOCAL citus.multi_shard_modify_mode TO on; + */ +void +SetLocalForceMaxQueryParallelization(void) +{ + set_config_option("citus.force_max_query_parallelization", "on", + (superuser() ? PGC_SUSET : PGC_USERSET), PGC_S_SESSION, + GUC_ACTION_LOCAL, true, 0, false); +} diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index ffce98bdb..85b9b6b1a 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -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. */ diff --git a/src/backend/distributed/executor/insert_select_executor.c b/src/backend/distributed/executor/insert_select_executor.c index 4c97ae8ad..4625ae6ab 100644 --- a/src/backend/distributed/executor/insert_select_executor.c +++ b/src/backend/distributed/executor/insert_select_executor.c @@ -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) diff --git a/src/backend/distributed/executor/multi_router_executor.c b/src/backend/distributed/executor/multi_router_executor.c index dac21c1e9..2e3176080 100644 --- a/src/backend/distributed/executor/multi_router_executor.c +++ b/src/backend/distributed/executor/multi_router_executor.c @@ -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) { diff --git a/src/backend/distributed/executor/multi_server_executor.c b/src/backend/distributed/executor/multi_server_executor.c index d4f3861bf..e54ca4358 100644 --- a/src/backend/distributed/executor/multi_server_executor.c +++ b/src/backend/distributed/executor/multi_server_executor.c @@ -20,6 +20,7 @@ #include #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) diff --git a/src/backend/distributed/master/master_create_shards.c b/src/backend/distributed/master/master_create_shards.c index 2ee64df33..42831105b 100644 --- a/src/backend/distributed/master/master_create_shards.c +++ b/src/backend/distributed/master/master_create_shards.c @@ -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(); + } } diff --git a/src/backend/distributed/master/master_stage_protocol.c b/src/backend/distributed/master/master_stage_protocol.c index b69f5f0cf..f6dbabaf1 100644 --- a/src/backend/distributed/master/master_stage_protocol.c +++ b/src/backend/distributed/master/master_stage_protocol.c @@ -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); diff --git a/src/backend/distributed/master/master_truncate.c b/src/backend/distributed/master/master_truncate.c index 938cfdda6..dff775e0a 100644 --- a/src/backend/distributed/master/master_truncate.c +++ b/src/backend/distributed/master/master_truncate.c @@ -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)); diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index b8a7c7072..88c705549 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -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; diff --git a/src/backend/distributed/planner/insert_select_planner.c b/src/backend/distributed/planner/insert_select_planner.c index 73165d4ab..6f097fe30 100644 --- a/src/backend/distributed/planner/insert_select_planner.c +++ b/src/backend/distributed/planner/insert_select_planner.c @@ -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) { diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index e4bd09cd8..1bd4cfaf3 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -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 diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 6856cd06c..e4a318f99 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -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, diff --git a/src/backend/distributed/transaction/remote_transaction.c b/src/backend/distributed/transaction/remote_transaction.c index 44e924a14..8213b31aa 100644 --- a/src/backend/distributed/transaction/remote_transaction.c +++ b/src/backend/distributed/transaction/remote_transaction.c @@ -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; } diff --git a/src/backend/distributed/utils/listutils.c b/src/backend/distributed/utils/listutils.c index 3643109b8..2d027c8eb 100644 --- a/src/backend/distributed/utils/listutils.c +++ b/src/backend/distributed/utils/listutils.c @@ -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; +} diff --git a/src/include/distributed/citus_custom_scan.h b/src/include/distributed/citus_custom_scan.h index 7d41c1340..3a01377d9 100644 --- a/src/include/distributed/citus_custom_scan.h +++ b/src/include/distributed/citus_custom_scan.h @@ -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; diff --git a/src/include/distributed/connection_management.h b/src/include/distributed/connection_management.h index bd5314ca1..d7396067e 100644 --- a/src/include/distributed/connection_management.h +++ b/src/include/distributed/connection_management.h @@ -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); diff --git a/src/include/distributed/listutils.h b/src/include/distributed/listutils.h index f08807190..22f8b65d6 100644 --- a/src/include/distributed/listutils.h +++ b/src/include/distributed/listutils.h @@ -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 */ diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 520be0a0f..9d2358a5c 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -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); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index 6fbab4915..8a573a079 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -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; /* diff --git a/src/include/distributed/multi_router_executor.h b/src/include/distributed/multi_router_executor.h index 17e7eb9c0..b92c46d30 100644 --- a/src/include/distributed/multi_router_executor.h +++ b/src/include/distributed/multi_router_executor.h @@ -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_ */ diff --git a/src/include/distributed/multi_server_executor.h b/src/include/distributed/multi_server_executor.h index 70b40aa83..d3ef25990 100644 --- a/src/include/distributed/multi_server_executor.h +++ b/src/include/distributed/multi_server_executor.h @@ -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; diff --git a/src/include/distributed/placement_connection.h b/src/include/distributed/placement_connection.h index 9cddc9d69..ab949066f 100644 --- a/src/include/distributed/placement_connection.h +++ b/src/include/distributed/placement_connection.h @@ -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 */ diff --git a/src/include/distributed/remote_transaction.h b/src/include/distributed/remote_transaction.h index 021fb80d5..69231a08f 100644 --- a/src/include/distributed/remote_transaction.h +++ b/src/include/distributed/remote_transaction.h @@ -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; diff --git a/src/test/regress/Makefile b/src/test/regress/Makefile index 3de412d8d..a8982b664 100644 --- a/src/test/regress/Makefile +++ b/src/test/regress/Makefile @@ -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) \ diff --git a/src/test/regress/bin/normalize.sed b/src/test/regress/bin/normalize.sed index a8463218d..12f764caa 100644 --- a/src/test/regress/bin/normalize.sed +++ b/src/test/regress/bin/normalize.sed @@ -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.*/\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 diff --git a/src/test/regress/bin/normalized_tests.lst b/src/test/regress/bin/normalized_tests.lst index f2bd35754..db95eb6d1 100644 --- a/src/test/regress/bin/normalized_tests.lst +++ b/src/test/regress/bin/normalized_tests.lst @@ -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 diff --git a/src/test/regress/expected/adaptive_executor.out b/src/test/regress/expected/adaptive_executor.out new file mode 100644 index 000000000..0da6193eb --- /dev/null +++ b/src/test/regress/expected/adaptive_executor.out @@ -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 diff --git a/src/test/regress/expected/failure_1pc_copy_append.out b/src/test/regress/expected/failure_1pc_copy_append.out index 810508d01..797ea7171 100644 --- a/src/test/regress/expected/failure_1pc_copy_append.out +++ b/src/test/regress/expected/failure_1pc_copy_append.out @@ -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; diff --git a/src/test/regress/expected/failure_1pc_copy_append_9.out b/src/test/regress/expected/failure_1pc_copy_append_9.out new file mode 100644 index 000000000..554d8baee --- /dev/null +++ b/src/test/regress/expected/failure_1pc_copy_append_9.out @@ -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; diff --git a/src/test/regress/expected/failure_1pc_copy_hash.out b/src/test/regress/expected/failure_1pc_copy_hash.out index 0354b7569..f888b2b43 100644 --- a/src/test/regress/expected/failure_1pc_copy_hash.out +++ b/src/test/regress/expected/failure_1pc_copy_hash.out @@ -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 diff --git a/src/test/regress/expected/failure_connection_establishment.out b/src/test/regress/expected/failure_connection_establishment.out index 4bf383963..6dc419197 100644 --- a/src/test/regress/expected/failure_connection_establishment.out +++ b/src/test/regress/expected/failure_connection_establishment.out @@ -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; diff --git a/src/test/regress/expected/failure_connection_establishment_9.out b/src/test/regress/expected/failure_connection_establishment_9.out new file mode 100644 index 000000000..bbb430e22 --- /dev/null +++ b/src/test/regress/expected/failure_connection_establishment_9.out @@ -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; diff --git a/src/test/regress/expected/failure_create_distributed_table_non_empty.out b/src/test/regress/expected/failure_create_distributed_table_non_empty.out index 794db86aa..0fa0d03e3 100644 --- a/src/test/regress/expected/failure_create_distributed_table_non_empty.out +++ b/src/test/regress/expected/failure_create_distributed_table_non_empty.out @@ -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 diff --git a/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out b/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out new file mode 100644 index 000000000..bea000678 --- /dev/null +++ b/src/test/regress/expected/failure_create_distributed_table_non_empty_9.out @@ -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; diff --git a/src/test/regress/expected/failure_create_reference_table_9.out b/src/test/regress/expected/failure_create_reference_table_9.out new file mode 100644 index 000000000..4c4ede61b --- /dev/null +++ b/src/test/regress/expected/failure_create_reference_table_9.out @@ -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; diff --git a/src/test/regress/expected/failure_create_table.out b/src/test/regress/expected/failure_create_table.out index 92a6c8a92..21b6e0d44 100644 --- a/src/test/regress/expected/failure_create_table.out +++ b/src/test/regress/expected/failure_create_table.out @@ -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()'); diff --git a/src/test/regress/expected/failure_create_table_9.out b/src/test/regress/expected/failure_create_table_9.out new file mode 100644 index 000000000..c1e3197b8 --- /dev/null +++ b/src/test/regress/expected/failure_create_table_9.out @@ -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; diff --git a/src/test/regress/expected/failure_cte_subquery_9.out b/src/test/regress/expected/failure_cte_subquery_9.out new file mode 100644 index 000000000..aba610d66 --- /dev/null +++ b/src/test/regress/expected/failure_cte_subquery_9.out @@ -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 diff --git a/src/test/regress/expected/failure_ddl.out b/src/test/regress/expected/failure_ddl.out index 27436dd56..8f2d23547 100644 --- a/src/test/regress/expected/failure_ddl.out +++ b/src/test/regress/expected/failure_ddl.out @@ -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 diff --git a/src/test/regress/expected/failure_ddl_9.out b/src/test/regress/expected/failure_ddl_9.out new file mode 100644 index 000000000..769e14b14 --- /dev/null +++ b/src/test/regress/expected/failure_ddl_9.out @@ -0,0 +1,1109 @@ +-- +-- Test DDL command propagation failures +-- Different dimensions we're testing: +-- Replication factor, 1PC-2PC, sequential-parallel modes +-- +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()'); + mitmproxy +----------- + +(1 row) + +SET citus.next_shard_id TO 100800; +-- we'll start with replication factor 1, 1PC and parallel mode +SET citus.multi_shard_commit_protocol TO '1pc'; +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 1; +CREATE TABLE test_table (key int, value int); +SELECT create_distributed_table('test_table', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +-- in the first test, kill just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- kill as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: connection error: localhost:9060 +DETAIL: server closed the connection unexpectedly + This probably means the server terminated abnormally + before or while processing the request. +-- show that we've never commited the changes +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +-- show that we've never commited the changes +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- kill as soon as the coordinator sends COMMIT +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- since we've killed the connection just after +-- the coordinator sends the COMMIT, the command should be applied +-- to the distributed table and the shards on the other worker +-- however, there is no way to recover the failure on the shards +-- that live in the failed worker, since we're running 1PC +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,value}") + (localhost,9060,100802,t,"{key,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- manually drop & re-create the table for the next tests +DROP TABLE test_table; +SET citus.next_shard_id TO 100800; +SET citus.multi_shard_commit_protocol TO '1pc'; +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 1; +CREATE TABLE test_table (key int, value int); +SELECT create_distributed_table('test_table', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +-- cancel as soon as the coordinator sends COMMIT +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- interrupts are held during COMMIT/ROLLBACK, so the command +-- should have been applied without any issues since cancel is ignored +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- the following tests rely the column not exists, so drop manually +ALTER TABLE test_table DROP COLUMN new_column; +-- but now kill just after the worker sends response to +-- COMMIT command, so we'll have lots of warnings but the command +-- should have been committed both on the distributed table and the placements +SET client_min_messages TO WARNING; +SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 +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 +WARNING: could not commit transaction for shard 100802 on any active node +WARNING: could not commit transaction for shard 100800 on any active node +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SET client_min_messages TO ERROR; +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- now cancel just after the worker sends response to +-- but Postgres doesn't accepts interrupts during COMMIT and ROLLBACK +-- so should not cancel at all, so not an effective test but adding in +-- case Citus messes up this behaviour +SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- the remaining tests rely on table having new_column +ALTER TABLE test_table ADD COLUMN new_column INT; +-- finally, test failing on ROLLBACK with 1PC +-- fail just after the coordinator sends the ROLLBACK +-- so the command can be rollbacked +SELECT citus.mitmproxy('conn.onQuery(query="ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +SET LOCAL client_min_messages TO WARNING; +ALTER TABLE test_table DROP COLUMN new_column; +ROLLBACK; +WARNING: connection not open +CONTEXT: while executing command on localhost:9060 +WARNING: connection not open +CONTEXT: while executing command on localhost:9060 +-- now cancel just after the worker sends response to +-- but Postgres doesn't accepts interrupts during COMMIT and ROLLBACK +-- so should not cancel at all, so not an effective test but adding in +-- case Citus messes up this behaviour +SELECT citus.mitmproxy('conn.onQuery(query="ROLLBACK").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +BEGIN; +ALTER TABLE test_table DROP COLUMN new_column; +ROLLBACK; +-- but now kill just after the worker sends response to +-- ROLLBACK command, so we'll have lots of warnings but the command +-- should have been rollbacked both on the distributed table and the placements +SELECT citus.mitmproxy('conn.onCommandComplete(command="ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +ALTER TABLE test_table DROP COLUMN new_column; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- now, lets test with 2PC +SET citus.multi_shard_commit_protocol TO '2pc'; +-- in the first test, kill just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +-- cancel just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +-- kill as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +-- cancel as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +-- killing on PREPARE should be fine, everything should be rollbacked +SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +ERROR: connection not open +CONTEXT: while executing command on localhost:9060 +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- we should be able to recover the transaction and +-- see that the command is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 2 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- cancelling on PREPARE should be fine, everything should be rollbacked +SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- we should be able to recover the transaction and +-- see that the command is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 1 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- killing on command complete of COMMIT PREPARE, we should see that the command succeeds +-- and all the workers committed +SELECT citus.mitmproxy('conn.onCommandComplete(command="COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +------------------------------------------ + (localhost,9060,100800,t,"{key,value}") + (localhost,9060,100802,t,"{key,value}") + (localhost,57637,100801,t,"{key,value}") + (localhost,57637,100803,t,"{key,value}") +(4 rows) + +-- we shouldn't have any prepared transactions in the workers +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +------------------------------------------ + (localhost,9060,100800,t,"{key,value}") + (localhost,9060,100802,t,"{key,value}") + (localhost,57637,100801,t,"{key,value}") + (localhost,57637,100803,t,"{key,value}") +(4 rows) + +-- kill as soon as the coordinator sends COMMIT +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- some of the placements would be missing the new column +-- since we've not commited the prepared transactions +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,value}") + (localhost,9060,100802,t,"{key,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- we should be able to recover the transaction and +-- see that the command is committed +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 2 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- finally, test failing on ROLLBACK with 2PC +-- fail just after the coordinator sends the ROLLBACK +-- so the command can be rollbacked +SELECT citus.mitmproxy('conn.onQuery(query="ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +ALTER TABLE test_table DROP COLUMN new_column; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- ROLLBACK should have failed on the distributed table and the placements +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- but now kill just after the worker sends response to +-- ROLLBACK command, so we'll have lots of warnings but the command +-- should have been rollbacked both on the distributed table and the placements +SELECT citus.mitmproxy('conn.onCommandComplete(command="ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +ALTER TABLE test_table DROP COLUMN new_column; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- make sure that the transaction is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100800,t,"{key,new_column,value}") + (localhost,9060,100802,t,"{key,new_column,value}") + (localhost,57637,100801,t,"{key,new_column,value}") + (localhost,57637,100803,t,"{key,new_column,value}") +(4 rows) + +-- another set of tests with 2PC and replication factor = 2 +SET citus.multi_shard_commit_protocol TO '2pc'; +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 2; +-- re-create the table with replication factor 2 +DROP TABLE test_table; +CREATE TABLE test_table (key int, value int); +SELECT create_distributed_table('test_table', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +-- in the first test, kill just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- kill as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- killing on PREPARE should be fine, everything should be rollbacked +SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: connection not open +CONTEXT: while executing command on localhost:9060 +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- we should be able to recover the transaction and +-- see that the command is rollbacked on all workers +-- note that in this case recover_prepared_transactions() +-- sends ROLLBACK PREPARED to the workers given that +-- the transaction has not been commited on any placement yet +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 4 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +------------------------------------------ + (localhost,9060,100804,t,"{key,value}") + (localhost,9060,100805,t,"{key,value}") + (localhost,9060,100806,t,"{key,value}") + (localhost,9060,100807,t,"{key,value}") + (localhost,57637,100804,t,"{key,value}") + (localhost,57637,100805,t,"{key,value}") + (localhost,57637,100806,t,"{key,value}") + (localhost,57637,100807,t,"{key,value}") +(8 rows) + +-- killing on command complete of COMMIT PREPARE, we should see that the command succeeds +-- and all the workers committed +SELECT citus.mitmproxy('conn.onCommandComplete(command="COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------------------ + {key,new_column,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100804,t,"{key,new_column,value}") + (localhost,9060,100805,t,"{key,new_column,value}") + (localhost,9060,100806,t,"{key,new_column,value}") + (localhost,9060,100807,t,"{key,new_column,value}") + (localhost,57637,100804,t,"{key,new_column,value}") + (localhost,57637,100805,t,"{key,new_column,value}") + (localhost,57637,100806,t,"{key,new_column,value}") + (localhost,57637,100807,t,"{key,new_column,value}") +(8 rows) + +-- we shouldn't have any prepared transactions in the workers +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +----------------------------------------------------- + (localhost,9060,100804,t,"{key,new_column,value}") + (localhost,9060,100805,t,"{key,new_column,value}") + (localhost,9060,100806,t,"{key,new_column,value}") + (localhost,9060,100807,t,"{key,new_column,value}") + (localhost,57637,100804,t,"{key,new_column,value}") + (localhost,57637,100805,t,"{key,new_column,value}") + (localhost,57637,100806,t,"{key,new_column,value}") + (localhost,57637,100807,t,"{key,new_column,value}") +(8 rows) + +-- kill as soon as the coordinator sends COMMIT +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table DROP COLUMN new_column; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- some of the placements would be missing the new column +-- since we've not commited the prepared transactions +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +---------------------------------------------------- + (localhost,9060,100804,t,"{key,new_column,value}") + (localhost,9060,100805,t,"{key,new_column,value}") + (localhost,9060,100806,t,"{key,new_column,value}") + (localhost,9060,100807,t,"{key,new_column,value}") + (localhost,57637,100804,t,"{key,value}") + (localhost,57637,100805,t,"{key,value}") + (localhost,57637,100806,t,"{key,value}") + (localhost,57637,100807,t,"{key,value}") +(8 rows) + +-- we should be able to recover the transaction and +-- see that the command is committed +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 4 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +------------------------------------------ + (localhost,9060,100804,t,"{key,value}") + (localhost,9060,100805,t,"{key,value}") + (localhost,9060,100806,t,"{key,value}") + (localhost,9060,100807,t,"{key,value}") + (localhost,57637,100804,t,"{key,value}") + (localhost,57637,100805,t,"{key,value}") + (localhost,57637,100806,t,"{key,value}") + (localhost,57637,100807,t,"{key,value}") +(8 rows) + +-- finally, test failing on ROLLBACK with 2PC +-- fail just after the coordinator sends the ROLLBACK +-- so the command can be rollbacked +SELECT citus.mitmproxy('conn.onQuery(query="ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +ALTER TABLE test_table ADD COLUMN new_column INT; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- ROLLBACK should have failed on the distributed table and the placements +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +------------------------------------------ + (localhost,9060,100804,t,"{key,value}") + (localhost,9060,100805,t,"{key,value}") + (localhost,9060,100806,t,"{key,value}") + (localhost,9060,100807,t,"{key,value}") + (localhost,57637,100804,t,"{key,value}") + (localhost,57637,100805,t,"{key,value}") + (localhost,57637,100806,t,"{key,value}") + (localhost,57637,100807,t,"{key,value}") +(8 rows) + +-- but now kill just after the worker sends response to +-- ROLLBACK command, so we'll have lots of warnings but the command +-- should have been rollbacked both on the distributed table and the placements +SELECT citus.mitmproxy('conn.onCommandComplete(command="ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +ALTER TABLE test_table ADD COLUMN new_column INT; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- make sure that the transaction is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT run_command_on_placements('test_table', $$SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = '%s'::regclass;$$) ORDER BY 1; + run_command_on_placements +------------------------------------------ + (localhost,9060,100804,t,"{key,value}") + (localhost,9060,100805,t,"{key,value}") + (localhost,9060,100806,t,"{key,value}") + (localhost,9060,100807,t,"{key,value}") + (localhost,57637,100804,t,"{key,value}") + (localhost,57637,100805,t,"{key,value}") + (localhost,57637,100806,t,"{key,value}") + (localhost,57637,100807,t,"{key,value}") +(8 rows) + +-- now do some tests with sequential mode +SET citus.multi_shard_modify_mode TO 'sequential'; +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SELECT array_agg(name::text ORDER BY name::text) FROM public.table_attrs where relid = 'test_table'::regclass; + array_agg +------------- + {key,value} +(1 row) + +-- kill as soon as the coordinator sends worker_apply_shard_ddl_command +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 as soon as the coordinator after it sends worker_apply_shard_ddl_command 2nd time +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").after(2).kill()'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +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 as soon as the coordinator after it sends worker_apply_shard_ddl_command 2nd time +SELECT citus.mitmproxy('conn.onQuery(query="worker_apply_shard_ddl_command").after(2).cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +ALTER TABLE test_table ADD COLUMN new_column INT; +ERROR: canceling statement due to user request +SET search_path TO 'public'; +DROP SCHEMA ddl_failure CASCADE; diff --git a/src/test/regress/expected/failure_insert_select_pushdown_9.out b/src/test/regress/expected/failure_insert_select_pushdown_9.out new file mode 100644 index 000000000..a27290627 --- /dev/null +++ b/src/test/regress/expected/failure_insert_select_pushdown_9.out @@ -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 diff --git a/src/test/regress/expected/failure_insert_select_via_coordinator_9.out b/src/test/regress/expected/failure_insert_select_via_coordinator_9.out new file mode 100644 index 000000000..735194327 --- /dev/null +++ b/src/test/regress/expected/failure_insert_select_via_coordinator_9.out @@ -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 diff --git a/src/test/regress/expected/failure_multi_dml_9.out b/src/test/regress/expected/failure_multi_dml_9.out new file mode 100644 index 000000000..6d8e928eb --- /dev/null +++ b/src/test/regress/expected/failure_multi_dml_9.out @@ -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; diff --git a/src/test/regress/expected/failure_multi_row_insert_9.out b/src/test/regress/expected/failure_multi_row_insert_9.out new file mode 100644 index 000000000..9b5570333 --- /dev/null +++ b/src/test/regress/expected/failure_multi_row_insert_9.out @@ -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 diff --git a/src/test/regress/expected/failure_multi_shard_update_delete.out b/src/test/regress/expected/failure_multi_shard_update_delete.out index ffcaeca16..853caa2f1 100644 --- a/src/test/regress/expected/failure_multi_shard_update_delete.out +++ b/src/test/regress/expected/failure_multi_shard_update_delete.out @@ -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 ----------- diff --git a/src/test/regress/expected/failure_multi_shard_update_delete_9.out b/src/test/regress/expected/failure_multi_shard_update_delete_9.out new file mode 100644 index 000000000..ff160802e --- /dev/null +++ b/src/test/regress/expected/failure_multi_shard_update_delete_9.out @@ -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 diff --git a/src/test/regress/expected/failure_real_time_select.out b/src/test/regress/expected/failure_real_time_select.out index 967569af2..89318e521 100644 --- a/src/test/regress/expected/failure_real_time_select.out +++ b/src/test/regress/expected/failure_real_time_select.out @@ -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 ------- diff --git a/src/test/regress/expected/failure_real_time_select_9.out b/src/test/regress/expected/failure_real_time_select_9.out new file mode 100644 index 000000000..2736cc9ca --- /dev/null +++ b/src/test/regress/expected/failure_real_time_select_9.out @@ -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; diff --git a/src/test/regress/expected/failure_ref_tables_9.out b/src/test/regress/expected/failure_ref_tables_9.out new file mode 100644 index 000000000..5727cb90c --- /dev/null +++ b/src/test/regress/expected/failure_ref_tables_9.out @@ -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; diff --git a/src/test/regress/expected/failure_savepoints.out b/src/test/regress/expected/failure_savepoints.out index aff797501..ccd6a4dfe 100644 --- a/src/test/regress/expected/failure_savepoints.out +++ b/src/test/regress/expected/failure_savepoints.out @@ -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 ----------- diff --git a/src/test/regress/expected/failure_savepoints_9.out b/src/test/regress/expected/failure_savepoints_9.out new file mode 100644 index 000000000..02163c4a3 --- /dev/null +++ b/src/test/regress/expected/failure_savepoints_9.out @@ -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; diff --git a/src/test/regress/expected/failure_single_mod_9.out b/src/test/regress/expected/failure_single_mod_9.out new file mode 100644 index 000000000..926682b7c --- /dev/null +++ b/src/test/regress/expected/failure_single_mod_9.out @@ -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; diff --git a/src/test/regress/expected/failure_single_select_9.out b/src/test/regress/expected/failure_single_select_9.out new file mode 100644 index 000000000..419777a00 --- /dev/null +++ b/src/test/regress/expected/failure_single_select_9.out @@ -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; diff --git a/src/test/regress/expected/failure_truncate.out b/src/test/regress/expected/failure_truncate.out index 82d90e90d..b9208ccac 100644 --- a/src/test/regress/expected/failure_truncate.out +++ b/src/test/regress/expected/failure_truncate.out @@ -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 diff --git a/src/test/regress/expected/failure_truncate_9.out b/src/test/regress/expected/failure_truncate_9.out new file mode 100644 index 000000000..4c86ff171 --- /dev/null +++ b/src/test/regress/expected/failure_truncate_9.out @@ -0,0 +1,1299 @@ +-- +-- Test TRUNCATE command failures +-- +CREATE SCHEMA truncate_failure; +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 +----------- + +(1 row) + +-- we'll start with replication factor 1, 1PC and parallel mode +SET citus.multi_shard_commit_protocol TO '1pc'; +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 1; +CREATE TABLE test_table (key int, value int); +SELECT create_distributed_table('test_table', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +CREATE VIEW unhealthy_shard_count AS + SELECT count(*) + FROM pg_dist_shard_placement pdsp + JOIN + pg_dist_shard pds + ON pdsp.shardid=pds.shardid + WHERE logicalrelid='truncate_failure.test_table'::regclass AND shardstate != 1; +-- in the first test, kill just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends TRUNCATE TABLE command +SELECT citus.mitmproxy('conn.onQuery(query="TRUNCATE TABLE truncate_failure.test_table").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel as soon as the coordinator sends TRUNCATE TABLE command +SELECT citus.mitmproxy('conn.onQuery(query="TRUNCATE TABLE truncate_failure.test_table").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends COMMIT +-- One shard should not get truncated but the other should +-- since it is sent from another connection. +-- Thus, we should see a partially successful truncate +-- Note: This is the result of using 1pc and there is no way to recover from it +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 8 +(1 row) + +-- refill the table +TRUNCATE test_table; +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- cancel as soon as the coordinator sends COMMIT +-- interrupts are held during COMMIT/ROLLBACK, so the command +-- should have been applied without any issues since cancel is ignored +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +-- refill the table +TRUNCATE test_table; +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +SET client_min_messages TO WARNING; +-- now kill just after the worker sends response to +-- COMMIT command, so we'll have lots of warnings but the command +-- should have been committed both on the distributed table and the placements +SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +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 +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 +WARNING: could not commit transaction for shard 120002 on any active node +WARNING: could not commit transaction for shard 120000 on any active node +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +SET client_min_messages TO ERROR; +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- now cancel just after the worker sends response to +-- but Postgres doesn't accept interrupts during COMMIT and ROLLBACK +-- so should not cancel at all, so not an effective test but adding in +-- case Citus messes up this behaviour +SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- Let's test Truncate on reference tables with a FK from a hash distributed table +CREATE TABLE reference_table(i int UNIQUE); +INSERT INTO reference_table SELECT x FROM generate_series(1,20) as f(x); +SELECT create_reference_table('reference_table'); + create_reference_table +------------------------ + +(1 row) + +ALTER TABLE test_table ADD CONSTRAINT foreign_key FOREIGN KEY (value) REFERENCES reference_table(i); +-- immediately kill when we see prepare transaction to see if the command +-- still cascaded to referencing table or failed successfuly +SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE reference_table CASCADE; +ERROR: connection not open +CONTEXT: while executing command on localhost:9060 +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +SELECT count(*) FROM reference_table; + count +------- + 20 +(1 row) + +-- immediately cancel when we see prepare transaction to see if the command +-- still cascaded to referencing table or failed successfuly +SELECT citus.mitmproxy('conn.onQuery(query="PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE reference_table CASCADE; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +SELECT count(*) FROM reference_table; + count +------- + 20 +(1 row) + +-- immediately kill when we see cascading TRUNCATE on the hash table to see +-- rollbacked properly +SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE TABLE").after(2).kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE reference_table CASCADE; +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 * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +SELECT count(*) FROM reference_table; + count +------- + 20 +(1 row) + +-- immediately cancel when we see cascading TRUNCATE on the hash table to see +-- if the command still cascaded to referencing table or failed successfuly +SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE TABLE").after(2).cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE reference_table CASCADE; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +SELECT count(*) FROM reference_table; + count +------- + 20 +(1 row) + +-- immediately kill after we get prepare transaction complete +-- to see if the command still cascaded to referencing table or +-- failed successfuly +SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE reference_table CASCADE; +ERROR: connection not open +CONTEXT: while executing command on localhost:9060 +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 1 +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- immediately cancel after we get prepare transaction complete +-- to see if the command still cascaded to referencing table or +-- failed successfuly +SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE reference_table CASCADE; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- now, lets test with 2PC +SET citus.multi_shard_commit_protocol TO '2pc'; +-- in the first test, kill just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends TRUNCATE TABLE command +SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE TABLE truncate_failure.test_table").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel as soon as the coordinator sends TRUNCATE TABLE command +SELECT citus.mitmproxy('conn.onQuery(query="^TRUNCATE TABLE truncate_failure.test_table").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- killing on PREPARE should be fine, everything should be rollbacked +SELECT citus.mitmproxy('conn.onCommandComplete(command="^PREPARE TRANSACTION").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: connection not open +CONTEXT: while executing command on localhost:9060 +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +-- we should be able to revocer the transaction and +-- see that the command is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 2 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancelling on PREPARE should be fine, everything should be rollbacked +SELECT citus.mitmproxy('conn.onCommandComplete(command="^PREPARE TRANSACTION").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +-- we should be able to revocer the transaction and +-- see that the command is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 1 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- killing on command complete of COMMIT PREPARE, we should see that the command succeeds +-- and all the workers committed +SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- we shouldn't have any prepared transactions in the workers +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- kill as soon as the coordinator sends COMMIT +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +-- Since we kill connections to one worker after commit arrives but the +-- other worker connections are healthy, we cannot commit on 1 worker +-- which has 2 active shard placements, but the other does. That's why +-- we expect to see 2 recovered prepared transactions. +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 2 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- finally, test failing on ROLLBACK with 2CPC +-- fail just after the coordinator sends the ROLLBACK +-- so the command can be rollbacked +SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +TRUNCATE test_table; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- but now kill just after the worker sends response to +-- ROLLBACK command, so we'll have lots of warnings but the command +-- should have been rollbacked both on the distributed table and the placements +SELECT citus.mitmproxy('conn.onCommandComplete(command="^ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +TRUNCATE test_table; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- final set of tests with 2PC and replication factor = 2 +SET citus.multi_shard_commit_protocol TO '2pc'; +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 2; +-- re-create the table with replication factor 2 +DROP TABLE test_table CASCADE; +CREATE TABLE test_table (key int, value int); +SELECT create_distributed_table('test_table', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +CREATE VIEW unhealthy_shard_count AS + SELECT count(*) + FROM pg_dist_shard_placement pdsp + JOIN + pg_dist_shard pds + ON pdsp.shardid=pds.shardid + WHERE logicalrelid='truncate_failure.test_table'::regclass AND shardstate != 1; +-- in the first test, kill just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel just in the first +-- response we get from the worker +SELECT citus.mitmproxy('conn.onAuthenticationOk().cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel as soon as the coordinator sends begin +SELECT citus.mitmproxy('conn.onQuery(query="^BEGIN TRANSACTION ISOLATION LEVEL READ COMMITTED").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- kill as soon as the coordinator sends TRUNCATE TABLE command +SELECT citus.mitmproxy('conn.onQuery(query="TRUNCATE TABLE truncate_failure.test_table").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE 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. +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- cancel as soon as the coordinator sends TRUNCATE TABLE command +SELECT citus.mitmproxy('conn.onQuery(query="TRUNCATE TABLE truncate_failure.test_table").cancel(' || pg_backend_pid() || ')'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: canceling statement due to user request +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- killing on PREPARE should be fine, everything should be rollbacked +SELECT citus.mitmproxy('conn.onCommandComplete(command="PREPARE TRANSACTION").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +ERROR: connection not open +CONTEXT: while executing command on localhost:9060 +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +-- we should be able to revocer the transaction and +-- see that the command is rollbacked +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 4 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- killing on command complete of COMMIT PREPARE, we should see that the command succeeds +-- and all the workers committed +SELECT citus.mitmproxy('conn.onCommandComplete(command="^COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +-- we shouldn't have any prepared transactions in the workers +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- kill as soon as the coordinator sends COMMIT +SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT PREPARED").kill()'); + mitmproxy +----------- + +(1 row) + +TRUNCATE test_table; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +-- Since we kill connections to one worker after commit arrives but the +-- other worker connections are healthy, we cannot commit on 1 worker +-- which has 4 active shard placements (2 shards, replication factor=2), +-- but the other does. That's why we expect to see 4 recovered prepared +-- transactions. +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 4 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 0 +(1 row) + +INSERT INTO test_table SELECT x,x FROM generate_series(1,20) as f(x); +-- finally, test failing on ROLLBACK with 2CPC +-- fail just after the coordinator sends the ROLLBACK +-- so the command can be rollbacked +SELECT citus.mitmproxy('conn.onQuery(query="^ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +TRUNCATE test_table; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +-- but now kill just after the worker sends response to +-- ROLLBACK command, so we'll have lots of warnings but the command +-- should have been rollbacked both on the distributed table and the placements +SELECT citus.mitmproxy('conn.onCommandComplete(command="^ROLLBACK").kill()'); + mitmproxy +----------- + +(1 row) + +BEGIN; +TRUNCATE test_table; +ROLLBACK; +SELECT citus.mitmproxy('conn.allow()'); + mitmproxy +----------- + +(1 row) + +SELECT * FROM unhealthy_shard_count; + count +------- + 0 +(1 row) + +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +SELECT count(*) FROM test_table; + count +------- + 20 +(1 row) + +DROP SCHEMA truncate_failure CASCADE; +SET search_path TO default; diff --git a/src/test/regress/expected/failure_vacuum.out b/src/test/regress/expected/failure_vacuum.out index d9a7ff19c..bab6490dd 100644 --- a/src/test/regress/expected/failure_vacuum.out +++ b/src/test/regress/expected/failure_vacuum.out @@ -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 diff --git a/src/test/regress/expected/failure_vacuum_0.out b/src/test/regress/expected/failure_vacuum_0.out index b4bfe5221..c4cba7a55 100644 --- a/src/test/regress/expected/failure_vacuum_0.out +++ b/src/test/regress/expected/failure_vacuum_0.out @@ -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 diff --git a/src/test/regress/expected/failure_vacuum_8.out b/src/test/regress/expected/failure_vacuum_8.out new file mode 100644 index 000000000..c5c6af40d --- /dev/null +++ b/src/test/regress/expected/failure_vacuum_8.out @@ -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; diff --git a/src/test/regress/expected/failure_vacuum_9.out b/src/test/regress/expected/failure_vacuum_9.out new file mode 100644 index 000000000..2b6510f62 --- /dev/null +++ b/src/test/regress/expected/failure_vacuum_9.out @@ -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; diff --git a/src/test/regress/expected/intermediate_results.out b/src/test/regress/expected/intermediate_results.out index 51d969020..f0abe1b10 100644 --- a/src/test/regress/expected/intermediate_results.out +++ b/src/test/regress/expected/intermediate_results.out @@ -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 diff --git a/src/test/regress/expected/isolation_citus_dist_activity.out b/src/test/regress/expected/isolation_citus_dist_activity.out index 1e197b4c0..e80ab62b2 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity.out +++ b/src/test/regress/expected/isolation_citus_dist_activity.out @@ -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; diff --git a/src/test/regress/expected/isolation_citus_dist_activity_0.out b/src/test/regress/expected/isolation_citus_dist_activity_9.out similarity index 50% rename from src/test/regress/expected/isolation_citus_dist_activity_0.out rename to src/test/regress/expected/isolation_citus_dist_activity_9.out index 067bf5bbb..c66d5b172 100644 --- a/src/test/regress/expected/isolation_citus_dist_activity_0.out +++ b/src/test/regress/expected/isolation_citus_dist_activity_9.out @@ -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; diff --git a/src/test/regress/expected/isolation_copy_placement_vs_modification.out b/src/test/regress/expected/isolation_copy_placement_vs_modification.out index 3afeb7689..4574cb055 100644 --- a/src/test/regress/expected/isolation_copy_placement_vs_modification.out +++ b/src/test/regress/expected/isolation_copy_placement_vs_modification.out @@ -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; diff --git a/src/test/regress/expected/isolation_distributed_transaction_id.out b/src/test/regress/expected/isolation_distributed_transaction_id.out index 211fc10fa..00d9d9e6c 100644 --- a/src/test/regress/expected/isolation_distributed_transaction_id.out +++ b/src/test/regress/expected/isolation_distributed_transaction_id.out @@ -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; diff --git a/src/test/regress/expected/isolation_dump_global_wait_edges.out b/src/test/regress/expected/isolation_dump_global_wait_edges.out index 9d3d7c2f5..5331a6b61 100644 --- a/src/test/regress/expected/isolation_dump_global_wait_edges.out +++ b/src/test/regress/expected/isolation_dump_global_wait_edges.out @@ -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; diff --git a/src/test/regress/expected/isolation_multi_shard_modify_vs_all.out b/src/test/regress/expected/isolation_multi_shard_modify_vs_all.out index d82e297eb..47411070d 100644 --- a/src/test/regress/expected/isolation_multi_shard_modify_vs_all.out +++ b/src/test/regress/expected/isolation_multi_shard_modify_vs_all.out @@ -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 diff --git a/src/test/regress/expected/isolation_replace_wait_function.out b/src/test/regress/expected/isolation_replace_wait_function.out index 48d2ef1ee..48681f023 100644 --- a/src/test/regress/expected/isolation_replace_wait_function.out +++ b/src/test/regress/expected/isolation_replace_wait_function.out @@ -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; diff --git a/src/test/regress/expected/limit_intermediate_size.out b/src/test/regress/expected/limit_intermediate_size.out index 044bc6c03..e54fd4c86 100644 --- a/src/test/regress/expected/limit_intermediate_size.out +++ b/src/test/regress/expected/limit_intermediate_size.out @@ -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; diff --git a/src/test/regress/expected/multi_binary_master_copy_format.out b/src/test/regress/expected/multi_binary_master_copy_format.out index 36643c014..8ff28f867 100644 --- a/src/test/regress/expected/multi_binary_master_copy_format.out +++ b/src/test/regress/expected/multi_binary_master_copy_format.out @@ -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 ------- diff --git a/src/test/regress/expected/multi_complex_expressions.out b/src/test/regress/expected/multi_complex_expressions.out index 6dd3dd5df..4321ea05f 100644 --- a/src/test/regress/expected/multi_complex_expressions.out +++ b/src/test/regress/expected/multi_complex_expressions.out @@ -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 diff --git a/src/test/regress/expected/multi_complex_expressions_0.out b/src/test/regress/expected/multi_complex_expressions_0.out index ddfe9dc58..eaf036a9f 100644 --- a/src/test/regress/expected/multi_complex_expressions_0.out +++ b/src/test/regress/expected/multi_complex_expressions_0.out @@ -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 diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index c4a31d298..a358ea3a5 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -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) diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 8834f4e11..d9c0f4ddf 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -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; diff --git a/src/test/regress/expected/multi_follower_task_tracker.out b/src/test/regress/expected/multi_follower_task_tracker.out index d60e39187..12d496867 100644 --- a/src/test/regress/expected/multi_follower_task_tracker.out +++ b/src/test/regress/expected/multi_follower_task_tracker.out @@ -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 ---+--- diff --git a/src/test/regress/expected/multi_function_in_join.out b/src/test/regress/expected/multi_function_in_join.out index c511a30bb..021aab809 100644 --- a/src/test/regress/expected/multi_function_in_join.out +++ b/src/test/regress/expected/multi_function_in_join.out @@ -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; diff --git a/src/test/regress/expected/multi_insert_select.out b/src/test/regress/expected/multi_insert_select.out index 43b21c207..a25403e60 100644 --- a/src/test/regress/expected/multi_insert_select.out +++ b/src/test/regress/expected/multi_insert_select.out @@ -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; diff --git a/src/test/regress/expected/multi_insert_select_9.out b/src/test/regress/expected/multi_insert_select_9.out new file mode 100644 index 000000000..3013f46fc --- /dev/null +++ b/src/test/regress/expected/multi_insert_select_9.out @@ -0,0 +1,2820 @@ +-- +-- MULTI_INSERT_SELECT +-- +SET citus.next_shard_id TO 13300000; +SET citus.next_placement_id TO 13300000; +-- create co-located tables +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 2; +-- order of execution might change in parallel executions +-- and the error details might contain the worker node +-- so be less verbose with \set VERBOSITY TERSE when necessary +CREATE TABLE raw_events_first (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint, UNIQUE(user_id, value_1)); +SELECT create_distributed_table('raw_events_first', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE raw_events_second (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint, UNIQUE(user_id, value_1)); +SELECT create_distributed_table('raw_events_second', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE agg_events (user_id int, value_1_agg int, value_2_agg int, value_3_agg float, value_4_agg bigint, agg_time timestamp, UNIQUE(user_id, value_1_agg)); +SELECT create_distributed_table('agg_events', 'user_id');; + create_distributed_table +-------------------------- + +(1 row) + +-- create the reference table as well +CREATE TABLE reference_table (user_id int); +SELECT create_reference_table('reference_table'); + create_reference_table +------------------------ + +(1 row) + +CREATE TABLE insert_select_varchar_test (key varchar, value int); +SELECT create_distributed_table('insert_select_varchar_test', 'key', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +-- set back to the defaults +SET citus.shard_count = DEFAULT; +SET citus.shard_replication_factor = DEFAULT; +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (1, now(), 10, 100, 1000.1, 10000); +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (2, now(), 20, 200, 2000.1, 20000); +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (3, now(), 30, 300, 3000.1, 30000); +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (4, now(), 40, 400, 4000.1, 40000); +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (5, now(), 50, 500, 5000.1, 50000); +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (6, now(), 60, 600, 6000.1, 60000); +SET client_min_messages TO DEBUG2; +-- raw table to raw table +INSERT INTO raw_events_second SELECT * FROM raw_events_first; +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- see that our first multi shard INSERT...SELECT works expected +SET client_min_messages TO INFO; +SELECT + raw_events_first.user_id +FROM + raw_events_first, raw_events_second +WHERE + raw_events_first.user_id = raw_events_second.user_id +ORDER BY + user_id DESC; + user_id +--------- + 6 + 5 + 4 + 3 + 2 + 1 +(6 rows) + +-- see that we get unique vialitons +\set VERBOSITY TERSE +INSERT INTO raw_events_second SELECT * FROM raw_events_first; +ERROR: duplicate key value violates unique constraint "raw_events_second_user_id_value_1_key_13300004" +\set VERBOSITY DEFAULT +-- stable functions should be allowed +INSERT INTO raw_events_second (user_id, time) +SELECT + user_id, now() +FROM + raw_events_first +WHERE + user_id < 0; +INSERT INTO raw_events_second (user_id) +SELECT + user_id +FROM + raw_events_first +WHERE + time > now() + interval '1 day'; +-- hide version-dependent PL/pgSQL context messages +\set VERBOSITY terse +-- make sure we evaluate stable functions on the master, once +CREATE OR REPLACE FUNCTION evaluate_on_master() +RETURNS int LANGUAGE plpgsql STABLE +AS $function$ +BEGIN + RAISE NOTICE 'evaluating on master'; + RETURN 0; +END; +$function$; +INSERT INTO raw_events_second (user_id, value_1) +SELECT + user_id, evaluate_on_master() +FROM + raw_events_first +WHERE + user_id < 0; +NOTICE: evaluating on master +-- make sure we don't evaluate stable functions with column arguments +CREATE OR REPLACE FUNCTION evaluate_on_master(x int) +RETURNS int LANGUAGE plpgsql STABLE +AS $function$ +BEGIN + RAISE NOTICE 'evaluating on master'; + RETURN x; +END; +$function$; +INSERT INTO raw_events_second (user_id, value_1) +SELECT + user_id, evaluate_on_master(value_1) +FROM + raw_events_first +WHERE + user_id = 0; +ERROR: function public.evaluate_on_master(integer) does not exist +-- add one more row +INSERT INTO raw_events_first (user_id, time) VALUES + (7, now()); +-- try a single shard query +SET client_min_messages TO DEBUG2; +INSERT INTO raw_events_second (user_id, time) SELECT user_id, time FROM raw_events_first WHERE user_id = 7; +DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned away +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, "time") SELECT user_id, "time" FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) 7) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +-- add one more row +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (8, now(), 80, 800, 8000, 80000); +-- reorder columns +SET client_min_messages TO DEBUG2; +INSERT INTO raw_events_second (value_2, value_1, value_3, value_4, user_id, time) +SELECT + value_2, value_1, value_3, value_4, user_id, time +FROM + raw_events_first +WHERE + user_id = 8; +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_first_13300000 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) 8) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away +DEBUG: Plan is router executable +-- a zero shard select +INSERT INTO raw_events_second (value_2, value_1, value_3, value_4, user_id, time) +SELECT + value_2, value_1, value_3, value_4, user_id, time +FROM + raw_events_first +WHERE + false; +DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away +DEBUG: Plan is router executable +-- another zero shard select +INSERT INTO raw_events_second (value_2, value_1, value_3, value_4, user_id, time) +SELECT + value_2, value_1, value_3, value_4, user_id, time +FROM + raw_events_first +WHERE + 0 != 0; +DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away +DEBUG: Plan is router executable +-- add one more row +SET client_min_messages TO INFO; +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (9, now(), 90, 900, 9000, 90000); +-- show that RETURNING also works +SET client_min_messages TO DEBUG2; +INSERT INTO raw_events_second (user_id, value_1, value_3) +SELECT + user_id, value_1, value_3 +FROM + raw_events_first +WHERE + value_3 = 9000 +RETURNING *; +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300000 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300001 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300002 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300003 raw_events_first WHERE ((value_3 OPERATOR(pg_catalog.=) (9000)::double precision) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: Plan is router executable + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+------+---------+---------+---------+--------- + 9 | | 90 | | 9000 | +(1 row) + +-- hits two shards +\set VERBOSITY TERSE +INSERT INTO raw_events_second (user_id, value_1, value_3) +SELECT + user_id, value_1, value_3 +FROM + raw_events_first +WHERE + user_id = 9 OR user_id = 16 +RETURNING *; +DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned away +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300001 raw_events_first WHERE (((user_id OPERATOR(pg_catalog.=) 9) OR (user_id OPERATOR(pg_catalog.=) 16)) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id, value_1, value_3) SELECT user_id, value_1, value_3 FROM public.raw_events_first_13300003 raw_events_first WHERE (((user_id OPERATOR(pg_catalog.=) 9) OR (user_id OPERATOR(pg_catalog.=) 16)) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: Plan is router executable +ERROR: duplicate key value violates unique constraint "raw_events_second_user_id_value_1_key_13300007" +-- now do some aggregations +INSERT INTO agg_events +SELECT + user_id, sum(value_1), avg(value_2), sum(value_3), count(value_4) +FROM + raw_events_first +GROUP BY + user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg, value_2_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, avg(value_2) AS avg, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY user_id +DEBUG: Plan is router executable +-- group by column not exists on the SELECT target list +INSERT INTO agg_events (value_3_agg, value_4_agg, value_1_agg, user_id) +SELECT + sum(value_3), count(value_4), sum(value_1), user_id +FROM + raw_events_first +GROUP BY + value_2, user_id +RETURNING *; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg, value_3_agg, value_4_agg) SELECT user_id, sum(value_1) AS sum, sum(value_3) AS sum, count(value_4) AS count FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY value_2, user_id RETURNING citus_table_alias.user_id, citus_table_alias.value_1_agg, citus_table_alias.value_2_agg, citus_table_alias.value_3_agg, citus_table_alias.value_4_agg, citus_table_alias.agg_time +DEBUG: Plan is router executable +ERROR: duplicate key value violates unique constraint "agg_events_user_id_value_1_agg_key_13300008" +-- some subquery tests +INSERT INTO agg_events + (value_1_agg, + user_id) +SELECT SUM(value_1), + id +FROM (SELECT raw_events_second.user_id AS id, + raw_events_second.value_1 + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id) AS foo +GROUP BY id +ORDER BY id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300000 raw_events_first, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY id ORDER BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY id ORDER BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY id ORDER BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT id, sum(value_1) AS sum FROM (SELECT raw_events_second.user_id AS id, raw_events_second.value_1 FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id)) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY id ORDER BY id +DEBUG: Plan is router executable +ERROR: duplicate key value violates unique constraint "agg_events_user_id_value_1_agg_key_13300008" +-- subquery one more level depth +INSERT INTO agg_events + (value_4_agg, + value_1_agg, + user_id) +SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id) AS foo +ORDER BY id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) ORDER BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) ORDER BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) ORDER BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg, value_4_agg) SELECT id, v1, v4 FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id) foo WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) ORDER BY id +DEBUG: Plan is router executable +ERROR: duplicate key value violates unique constraint "agg_events_user_id_value_1_agg_key_13300008" +\set VERBOSITY DEFAULT +-- join between subqueries +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id); +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- add one more level subqueris on top of subquery JOINs +INSERT INTO agg_events + (user_id, value_4_agg) +SELECT + outer_most.id, max(outer_most.value) +FROM +( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id)) as outer_most +GROUP BY + outer_most.id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY id +DEBUG: Plan is router executable +-- subqueries in WHERE clause +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT user_id + FROM raw_events_second + WHERE user_id = 2); +DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) 2))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT user_id + FROM raw_events_second + WHERE user_id != 2 AND value_1 = 2000) +ON conflict (user_id, value_1) DO NOTHING; +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300000 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300004 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) ON CONFLICT(user_id, value_1) DO NOTHING +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) ON CONFLICT(user_id, value_1) DO NOTHING +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) ON CONFLICT(user_id, value_1) DO NOTHING +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.<>) 2) AND (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) ON CONFLICT(user_id, value_1) DO NOTHING +DEBUG: Plan is router executable +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT user_id + FROM raw_events_second WHERE false); +DEBUG: Skipping target shard interval 13300004 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away +DEBUG: Plan is router executable +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT user_id + FROM raw_events_second + WHERE value_1 = 1000 OR value_1 = 2000 OR value_1 = 3000); +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300000 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300004 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second WHERE ((raw_events_second.value_1 OPERATOR(pg_catalog.=) 1000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 2000) OR (raw_events_second.value_1 OPERATOR(pg_catalog.=) 3000)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable +-- lets mix subqueries in FROM clause and subqueries in WHERE +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 1000) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second); +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300004 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.reference_table_13300012 reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (1000)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE ((f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second)) AND ((worker_hash(f2.id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(f2.id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable +-- some UPSERTS +INSERT INTO agg_events AS ae + ( + user_id, + value_1_agg, + agg_time + ) +SELECT user_id, + value_1, + time +FROM raw_events_first +ON conflict (user_id, value_1_agg) +DO UPDATE + SET agg_time = EXCLUDED.agg_time + WHERE ae.agg_time < EXCLUDED.agg_time; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) +DEBUG: Plan is router executable +-- upserts with returning +INSERT INTO agg_events AS ae + ( + user_id, + value_1_agg, + agg_time + ) +SELECT user_id, + value_1, + time +FROM raw_events_first +ON conflict (user_id, value_1_agg) +DO UPDATE + SET agg_time = EXCLUDED.agg_time + WHERE ae.agg_time < EXCLUDED.agg_time +RETURNING user_id, value_1_agg; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS ae (user_id, value_1_agg, agg_time) SELECT user_id, value_1, "time" FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) ON CONFLICT(user_id, value_1_agg) DO UPDATE SET agg_time = excluded.agg_time WHERE (ae.agg_time OPERATOR(pg_catalog.<) excluded.agg_time) RETURNING ae.user_id, ae.value_1_agg +DEBUG: Plan is router executable + user_id | value_1_agg +---------+------------- + 7 | +(1 row) + +INSERT INTO agg_events (user_id, value_1_agg) +SELECT + user_id, sum(value_1 + value_2) +FROM + raw_events_first GROUP BY user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) AS sum FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY user_id +DEBUG: Plan is router executable +-- FILTER CLAUSE +INSERT INTO agg_events (user_id, value_1_agg) +SELECT + user_id, sum(value_1 + value_2) FILTER (where value_3 = 15) +FROM + raw_events_first GROUP BY user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT user_id, sum((value_1 OPERATOR(pg_catalog.+) value_2)) FILTER (WHERE (value_3 OPERATOR(pg_catalog.=) (15)::double precision)) AS sum FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY user_id +DEBUG: Plan is router executable +-- a test with reference table JOINs +INSERT INTO + agg_events (user_id, value_1_agg) +SELECT + raw_events_first.user_id, sum(value_1) +FROM + reference_table, raw_events_first +WHERE + raw_events_first.user_id = reference_table.user_id +GROUP BY + raw_events_first.user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300000 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) GROUP BY raw_events_first.user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300001 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) GROUP BY raw_events_first.user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300002 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823))) GROUP BY raw_events_first.user_id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT raw_events_first.user_id, sum(raw_events_first.value_1) AS sum FROM public.reference_table_13300012 reference_table, public.raw_events_first_13300003 raw_events_first WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647))) GROUP BY raw_events_first.user_id +DEBUG: Plan is router executable +-- a note on the outer joins is that +-- we filter out outer join results +-- where partition column returns +-- NULL. Thus, we could INSERT less rows +-- than we expect from subquery result. +-- see the following tests +SET client_min_messages TO INFO; +-- we don't want to see constraint vialotions, so truncate first +TRUNCATE agg_events; +-- add a row to first table to make table contents different +INSERT INTO raw_events_second (user_id, time, value_1, value_2, value_3, value_4) VALUES + (10, now(), 100, 10000, 10000, 100000); +DELETE FROM raw_events_second WHERE user_id = 2; +-- we select 11 rows +SELECT t1.user_id AS col1, + t2.user_id AS col2 + FROM raw_events_first t1 + FULL JOIN raw_events_second t2 + ON t1.user_id = t2.user_id + ORDER BY t1.user_id, + t2.user_id; + col1 | col2 +------+------ + 1 | 1 + 2 | + 3 | 3 + 4 | 4 + 5 | 5 + 6 | 6 + 7 | 7 + 8 | 8 + 9 | 9 + | 10 +(10 rows) + +SET client_min_messages TO DEBUG2; +-- we insert 10 rows since we filtered out +-- NULL partition column values +INSERT INTO agg_events (user_id, value_1_agg) +SELECT t1.user_id AS col1, + t2.user_id AS col2 +FROM raw_events_first t1 + FULL JOIN raw_events_second t2 + ON t1.user_id = t2.user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300000 t1 FULL JOIN public.raw_events_second_13300004 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300001 t1 FULL JOIN public.raw_events_second_13300005 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300002 t1 FULL JOIN public.raw_events_second_13300006 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT t1.user_id AS col1, t2.user_id AS col2 FROM (public.raw_events_first_13300003 t1 FULL JOIN public.raw_events_second_13300007 t2 ON ((t1.user_id OPERATOR(pg_catalog.=) t2.user_id))) WHERE ((worker_hash(t1.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(t1.user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +-- see that the results are different from the SELECT query +SELECT + user_id, value_1_agg +FROM + agg_events +ORDER BY + user_id, value_1_agg; + user_id | value_1_agg +---------+------------- + 1 | 1 + 2 | + 3 | 3 + 4 | 4 + 5 | 5 + 6 | 6 + 7 | 7 + 8 | 8 + 9 | 9 +(9 rows) + +-- we don't want to see constraint vialotions, so truncate first +SET client_min_messages TO INFO; +TRUNCATE agg_events; +SET client_min_messages TO DEBUG2; +-- DISTINCT clause +INSERT INTO agg_events (value_1_agg, user_id) + SELECT + DISTINCT value_1, user_id + FROM + raw_events_first; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT user_id, value_1 FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- we don't want to see constraint vialotions, so truncate first +SET client_min_messages TO INFO; +truncate agg_events; +SET client_min_messages TO DEBUG2; +-- DISTINCT ON clauses are supported +-- distinct on(non-partition column) +-- values are pulled to master +INSERT INTO agg_events (value_1_agg, user_id) + SELECT + DISTINCT ON (value_1) value_1, user_id + FROM + raw_events_first; +DEBUG: DISTINCT ON (non-partition column) clauses are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2; +DEBUG: Router planner cannot handle multi-shard select queries + user_id | value_1_agg +---------+------------- + 1 | 10 + 2 | 20 + 3 | 30 + 4 | 40 + 5 | 50 + 6 | 60 + 7 | + 8 | 80 + 9 | 90 +(9 rows) + +-- we don't want to see constraint vialotions, so truncate first +SET client_min_messages TO INFO; +truncate agg_events; +SET client_min_messages TO DEBUG2; +-- distinct on(partition column) +-- queries are forwared to workers +INSERT INTO agg_events (value_1_agg, user_id) + SELECT + DISTINCT ON (user_id) value_1, user_id + FROM + raw_events_first; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300000 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300001 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300002 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_1_agg) SELECT DISTINCT ON (user_id) user_id, value_1 FROM public.raw_events_first_13300003 raw_events_first WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +SELECT user_id, value_1_agg FROM agg_events ORDER BY 1,2; +DEBUG: Router planner cannot handle multi-shard select queries + user_id | value_1_agg +---------+------------- + 1 | 10 + 2 | 20 + 3 | 30 + 4 | 40 + 5 | 50 + 6 | 60 + 7 | + 8 | 80 + 9 | 90 +(9 rows) + +-- We support CTEs +BEGIN; +WITH fist_table_agg AS + (SELECT max(value_1)+1 as v1_agg, user_id FROM raw_events_first GROUP BY user_id) +INSERT INTO agg_events + (value_1_agg, user_id) + SELECT + v1_agg, user_id + FROM + fist_table_agg; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: generating subplan 51_1 for CTE fist_table_agg: SELECT (max(value_1) OPERATOR(pg_catalog.+) 1) AS v1_agg, user_id FROM public.raw_events_first GROUP BY user_id +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Plan 51 query after replacing subqueries and CTEs: SELECT user_id, v1_agg FROM (SELECT fist_table_agg.v1_agg, fist_table_agg.user_id FROM (SELECT intermediate_result.v1_agg, intermediate_result.user_id FROM read_intermediate_result('51_1'::text, 'binary'::citus_copy_format) intermediate_result(v1_agg integer, user_id integer)) fist_table_agg) citus_insert_select_subquery +DEBUG: Creating router plan +DEBUG: Plan is router executable +ROLLBACK; +-- We don't support CTEs that are referenced in the target list +INSERT INTO agg_events + WITH sub_cte AS (SELECT 1) + SELECT + raw_events_first.user_id, (SELECT * FROM sub_cte) + FROM + raw_events_first; +DEBUG: Subqueries without relations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: generating subplan 54_1 for CTE sub_cte: SELECT 1 +DEBUG: Plan 54 query after replacing subqueries and CTEs: SELECT user_id, (SELECT sub_cte."?column?" FROM (SELECT intermediate_result."?column?" FROM read_intermediate_result('54_1'::text, 'binary'::citus_copy_format) intermediate_result("?column?" integer)) sub_cte) FROM public.raw_events_first +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: could not run distributed query with subquery outside the FROM and WHERE clauses +HINT: Consider using an equality filter on the distributed table's partition column. +-- We support set operations via the coordinator +BEGIN; +INSERT INTO + raw_events_first(user_id) +SELECT + user_id +FROM + ((SELECT user_id FROM raw_events_first) UNION + (SELECT user_id FROM raw_events_second)) as foo; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ROLLBACK; +-- We do support set operations through recursive planning +BEGIN; +SET LOCAL client_min_messages TO DEBUG; +INSERT INTO + raw_events_first(user_id) + (SELECT user_id FROM raw_events_first) INTERSECT + (SELECT user_id FROM raw_events_first); +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: generating subplan 58_1 for subquery SELECT user_id FROM public.raw_events_first +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: generating subplan 58_2 for subquery SELECT user_id FROM public.raw_events_first +DEBUG: Creating router plan +DEBUG: Plan is router executable +DEBUG: generating subplan 58_3 for subquery SELECT intermediate_result.user_id FROM read_intermediate_result('58_1'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) INTERSECT SELECT intermediate_result.user_id FROM read_intermediate_result('58_2'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer) +DEBUG: Plan 58 query after replacing subqueries and CTEs: SELECT user_id FROM (SELECT intermediate_result.user_id FROM read_intermediate_result('58_3'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer)) citus_insert_select_subquery +DEBUG: Creating router plan +DEBUG: Plan is router executable +ROLLBACK; +-- If the query is router plannable then it is executed via the coordinator +INSERT INTO + raw_events_first(user_id) +SELECT + user_id +FROM + ((SELECT user_id FROM raw_events_first WHERE user_id = 15) EXCEPT + (SELECT user_id FROM raw_events_second where user_id = 17)) as foo; +DEBUG: Set operations are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Creating router plan +DEBUG: Plan is router executable +-- some supported LEFT joins + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300000 raw_events_first LEFT JOIN public.raw_events_second_13300004 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN public.raw_events_second_13300005 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN public.raw_events_second_13300006 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300003 raw_events_first LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_second.user_id + FROM + reference_table LEFT JOIN raw_events_second ON reference_table.user_id = raw_events_second.user_id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300004 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300005 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300006 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_second.user_id FROM (public.reference_table_13300012 reference_table LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((reference_table.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_second.user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_first.user_id = 10; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300000 raw_events_first LEFT JOIN public.raw_events_second_13300004 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) 10) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: Skipping target shard interval 13300009 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300010 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300011 since SELECT query for it pruned away +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_second.user_id = 10 OR raw_events_second.user_id = 11; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300000 raw_events_first LEFT JOIN public.raw_events_second_13300004 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN (SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_second(user_id, "time", value_1, value_2, value_3, value_4) ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN (SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_second(user_id, "time", value_1, value_2, value_3, value_4) ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300003 raw_events_first LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE (((raw_events_second.user_id OPERATOR(pg_catalog.=) 10) OR (raw_events_second.user_id OPERATOR(pg_catalog.=) 11)) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_first.user_id = 10 AND raw_events_first.user_id = 20; +DEBUG: Skipping target shard interval 13300008 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300009 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300010 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300011 since SELECT query for it pruned away +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_first.user_id = 10 AND raw_events_second.user_id = 20; +DEBUG: Skipping target shard interval 13300008 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300009 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300010 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300011 since SELECT query for it pruned away +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_first.user_id IN (19, 20, 21); +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300000 raw_events_first LEFT JOIN public.raw_events_second_13300004 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN public.raw_events_second_13300005 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN public.raw_events_second_13300006 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM ((SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_first(user_id, "time", value_1, value_2, value_3, value_4) LEFT JOIN public.raw_events_second_13300007 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_first.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable + + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.user_id + WHERE raw_events_second.user_id IN (19, 20, 21); +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300000 raw_events_first JOIN public.raw_events_second_13300004 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300001 raw_events_first JOIN public.raw_events_second_13300005 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300002 raw_events_first JOIN public.raw_events_second_13300006 raw_events_second ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id) SELECT raw_events_first.user_id FROM (public.raw_events_first_13300003 raw_events_first JOIN (SELECT NULL::integer AS user_id, NULL::timestamp without time zone AS "time", NULL::integer AS value_1, NULL::integer AS value_2, NULL::double precision AS value_3, NULL::bigint AS value_4 WHERE false) raw_events_second(user_id, "time", value_1, value_2, value_3, value_4) ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id))) WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) ANY (ARRAY[19, 20, 21])) AND ((worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(raw_events_first.user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable + + -- the following is a very tricky query for Citus + -- although we do not support pushing down JOINs on non-partition + -- columns here it is safe to push it down given that we're looking for + -- a specific value (i.e., value_1 = 12) on the joining column. + -- Note that the query always hits the same shard on raw_events_second + -- and this query wouldn't have worked if we're to use different worker + -- count or shard replication factor + INSERT INTO agg_events + (user_id) + SELECT raw_events_first.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_second.user_id = raw_events_first.value_1 + AND raw_events_first.value_1 = 12; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- some unsupported LEFT/INNER JOINs + -- JOIN on one table with partition column other is not + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- same as the above with INNER JOIN + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- a not meaningful query + INSERT INTO agg_events + (user_id) + SELECT raw_events_second.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_first.value_1; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- both tables joined on non-partition columns + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.value_1 = raw_events_second.value_1; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- same as the above with INNER JOIN + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first INNER JOIN raw_events_second ON raw_events_first.value_1 = raw_events_second.value_1; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + +-- even if there is a filter on the partition key, since the join is not on the partition key we reject +-- this query +INSERT INTO agg_events (user_id) +SELECT + raw_events_first.user_id +FROM + raw_events_first LEFT JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 +WHERE + raw_events_first.user_id = 10; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- same as the above with INNER JOIN + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 + WHERE raw_events_first.user_id = 10; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- make things a bit more complicate with IN clauses + INSERT INTO agg_events (user_id) + SELECT + raw_events_first.user_id + FROM + raw_events_first INNER JOIN raw_events_second ON raw_events_first.user_id = raw_events_second.value_1 + WHERE raw_events_first.value_1 IN (10, 11,12) OR raw_events_second.user_id IN (1,2,3,4); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- implicit join on non partition column should also not be pushed down + INSERT INTO agg_events + (user_id) + SELECT raw_events_first.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_second.user_id = raw_events_first.value_1; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- the following is again a tricky query for Citus + -- if the given filter was on value_1 as shown in the above, Citus could + -- push it down. But here the query is refused + INSERT INTO agg_events + (user_id) + SELECT raw_events_first.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_second.user_id = raw_events_first.value_1 + AND raw_events_first.value_2 = 12; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + + -- lets do some unsupported query tests with subqueries + -- foo is not joined on the partition key so the query is not + -- pushed down + INSERT INTO agg_events + (user_id, value_4_agg) + SELECT + outer_most.id, max(outer_most.value) + FROM + ( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first LEFT JOIN + reference_table + ON (raw_events_first.value_1 = reference_table.user_id)) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 + ON (f.id = f2.id)) as outer_most + GROUP BY + outer_most.id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + -- if the given filter was on value_1 as shown in the above, Citus could + -- push it down. But here the query is refused + INSERT INTO agg_events + (user_id) + SELECT raw_events_first.user_id + FROM raw_events_first, + raw_events_second + WHERE raw_events_second.user_id = raw_events_first.value_1 + AND raw_events_first.value_2 = 12; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. + -- lets do some unsupported query tests with subqueries + -- foo is not joined on the partition key so the query is not + -- pushed down + INSERT INTO agg_events + (user_id, value_4_agg) + SELECT + outer_most.id, max(outer_most.value) + FROM + ( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first LEFT JOIN + reference_table + ON (raw_events_first.value_1 = reference_table.user_id)) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 + ON (f.id = f2.id)) as outer_most + GROUP BY + outer_most.id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +INSERT INTO agg_events + (value_4_agg, + value_1_agg, + user_id) +SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id != raw_events_second.user_id + GROUP BY raw_events_second.user_id) AS foo; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- INSERT returns NULL partition key value via coordinator +INSERT INTO agg_events + (value_4_agg, + value_1_agg, + user_id) +SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.value_3 AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.value_3) AS foo; +DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1] +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [0,1073741823] +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [1073741824,2147483647] +DEBUG: join prunable for intervals [-1073741824,-1] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [-1073741824,-1] and [0,1073741823] +DEBUG: join prunable for intervals [-1073741824,-1] and [1073741824,2147483647] +DEBUG: join prunable for intervals [0,1073741823] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [0,1073741823] and [-1073741824,-1] +DEBUG: join prunable for intervals [0,1073741823] and [1073741824,2147483647] +DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1] +DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823] +DEBUG: generating subplan 88_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.value_3 AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_3 +DEBUG: Plan 88 query after replacing subqueries and CTEs: SELECT id, v1, v4 FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('88_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 bigint, id double precision)) foo +DEBUG: Creating router plan +DEBUG: Plan is router executable +ERROR: the partition column of table public.agg_events cannot be NULL +-- error cases +-- no part column at all +INSERT INTO raw_events_second + (value_1) +SELECT value_1 +FROM raw_events_first; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: the query doesn't include the target table's partition column +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: the partition column of table public.raw_events_second should have a value +INSERT INTO raw_events_second + (value_1) +SELECT user_id +FROM raw_events_first; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: the query doesn't include the target table's partition column +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: the partition column of table public.raw_events_second should have a value +INSERT INTO raw_events_second + (user_id) +SELECT value_1 +FROM raw_events_first; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: The target table's partition column should correspond to a partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: the partition column of table public.raw_events_second cannot be NULL +INSERT INTO raw_events_second + (user_id) +SELECT user_id * 2 +FROM raw_events_first; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an operator in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO raw_events_second + (user_id) +SELECT user_id :: bigint +FROM raw_events_first; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an explicit cast in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO agg_events + (value_3_agg, + value_4_agg, + value_1_agg, + value_2_agg, + user_id) +SELECT SUM(value_3), + Count(value_4), + user_id, + SUM(value_1), + Avg(value_2) +FROM raw_events_first +GROUP BY user_id; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an aggregation in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: the partition column of table public.agg_events cannot be NULL +INSERT INTO agg_events + (value_3_agg, + value_4_agg, + value_1_agg, + value_2_agg, + user_id) +SELECT SUM(value_3), + Count(value_4), + user_id, + SUM(value_1), + value_2 +FROM raw_events_first +GROUP BY user_id, + value_2; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: The target table's partition column should correspond to a partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: the partition column of table public.agg_events cannot be NULL +-- tables should be co-located +INSERT INTO agg_events (user_id) +SELECT + user_id +FROM + reference_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: The target table's partition column should correspond to a partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Distributed planning for a fast-path router query +DEBUG: Creating router plan +DEBUG: Plan is router executable +-- foo2 is recursively planned and INSERT...SELECT is done via coordinator +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + raw_events_second.value_1 AS v1, + SUM(raw_events_second.user_id) AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.value_1 + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id); +DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1] +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [0,1073741823] +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [1073741824,2147483647] +DEBUG: join prunable for intervals [-1073741824,-1] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [-1073741824,-1] and [0,1073741823] +DEBUG: join prunable for intervals [-1073741824,-1] and [1073741824,2147483647] +DEBUG: join prunable for intervals [0,1073741823] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [0,1073741823] and [-1073741824,-1] +DEBUG: join prunable for intervals [0,1073741823] and [1073741824,2147483647] +DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1] +DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823] +DEBUG: generating subplan 107_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric) +DEBUG: Plan 107 query after replacing subqueries and CTEs: SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('107_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) +DEBUG: Router planner cannot handle multi-shard select queries +-- the second part of the query is not routable since +-- GROUP BY not on the partition column (i.e., value_1) and thus join +-- on f.id = f2.id is not on the partition key (instead on the sum of partition key) +-- but we still recursively plan foo2 and run the query +INSERT INTO agg_events + (user_id) +SELECT f.id FROM +(SELECT + id +FROM (SELECT raw_events_first.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + raw_events_second.value_1 AS v1, + SUM(raw_events_second.user_id) AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.value_1 + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id); +DEBUG: Group by list without distribution column is not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: Router planner cannot handle multi-shard select queries +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [-1073741824,-1] +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [0,1073741823] +DEBUG: join prunable for intervals [-2147483648,-1073741825] and [1073741824,2147483647] +DEBUG: join prunable for intervals [-1073741824,-1] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [-1073741824,-1] and [0,1073741823] +DEBUG: join prunable for intervals [-1073741824,-1] and [1073741824,2147483647] +DEBUG: join prunable for intervals [0,1073741823] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [0,1073741823] and [-1073741824,-1] +DEBUG: join prunable for intervals [0,1073741823] and [1073741824,2147483647] +DEBUG: join prunable for intervals [1073741824,2147483647] and [-2147483648,-1073741825] +DEBUG: join prunable for intervals [1073741824,2147483647] and [-1073741824,-1] +DEBUG: join prunable for intervals [1073741824,2147483647] and [0,1073741823] +DEBUG: generating subplan 110_1 for subquery SELECT sum(raw_events_second.value_4) AS v4, raw_events_second.value_1 AS v1, sum(raw_events_second.user_id) AS id FROM public.raw_events_first, public.raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.value_1 HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric) +DEBUG: Plan 110 query after replacing subqueries and CTEs: SELECT f.id FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM public.raw_events_first, public.reference_table WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT intermediate_result.v4, intermediate_result.v1, intermediate_result.id FROM read_intermediate_result('110_1'::text, 'binary'::citus_copy_format) intermediate_result(v4 numeric, v1 integer, id bigint)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) +DEBUG: Router planner cannot handle multi-shard select queries +-- cannot pushdown the query since the JOIN is not equi JOIN +INSERT INTO agg_events + (user_id, value_4_agg) +SELECT +outer_most.id, max(outer_most.value) + FROM +( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id != f2.id)) as outer_most +GROUP BY outer_most.id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- cannot pushdown since foo2 is not join on partition key +INSERT INTO agg_events + (user_id, value_4_agg) +SELECT + outer_most.id, max(outer_most.value) +FROM +( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.value_1 + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id)) as outer_most +GROUP BY + outer_most.id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- cannot push down since foo doesn't have en equi join +INSERT INTO agg_events + (user_id, value_4_agg) +SELECT + outer_most.id, max(outer_most.value) +FROM +( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id != reference_table.user_id ) AS foo) as f + INNER JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id)) as outer_most +GROUP BY + outer_most.id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- some unsupported LATERAL JOINs +-- join on averages is not on the partition key +INSERT INTO agg_events (user_id, value_4_agg) +SELECT + averages.user_id, avg(averages.value_4) +FROM + (SELECT + raw_events_second.user_id + FROM + reference_table JOIN raw_events_second on (reference_table.user_id = raw_events_second.user_id) + ) reference_ids + JOIN LATERAL + (SELECT + user_id, value_4 + FROM + raw_events_first WHERE + value_4 = reference_ids.user_id) as averages ON true + GROUP BY averages.user_id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- join among reference_ids and averages is not on the partition key +INSERT INTO agg_events (user_id, value_4_agg) +SELECT + averages.user_id, avg(averages.value_4) +FROM + (SELECT + raw_events_second.user_id + FROM + reference_table JOIN raw_events_second on (reference_table.user_id = raw_events_second.user_id) + ) reference_ids + JOIN LATERAL + (SELECT + user_id, value_4 + FROM + raw_events_first) as averages ON averages.value_4 = reference_ids.user_id + GROUP BY averages.user_id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- join among the agg_ids and averages is not on the partition key +INSERT INTO agg_events (user_id, value_4_agg) +SELECT + averages.user_id, avg(averages.value_4) +FROM + (SELECT + raw_events_second.user_id + FROM + reference_table JOIN raw_events_second on (reference_table.user_id = raw_events_second.user_id) + ) reference_ids + JOIN LATERAL + (SELECT + user_id, value_4 + FROM + raw_events_first) as averages ON averages.user_id = reference_ids.user_id +JOIN LATERAL + (SELECT user_id, value_4 FROM agg_events) as agg_ids ON (agg_ids.value_4 = averages.user_id) + GROUP BY averages.user_id; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- not supported subqueries in WHERE clause +-- since the selected value in the WHERE is not +-- partition key +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT value_1 + FROM raw_events_second); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- same as above but slightly more complex +-- since it also includes subquery in FROM as well +INSERT INTO agg_events + (user_id) +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT value_1 + FROM raw_events_second); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- some more semi-anti join tests +-- join in where +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN (SELECT raw_events_second.user_id + FROM raw_events_second, raw_events_first + WHERE raw_events_second.user_id = raw_events_first.user_id AND raw_events_first.user_id = 200); +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300000 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300004 raw_events_second, public.raw_events_first_13300000 raw_events_first_1 WHERE ((raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first_1.user_id) AND (raw_events_first_1.user_id OPERATOR(pg_catalog.=) 200)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: Skipping target shard interval 13300005 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300006 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300007 since SELECT query for it pruned away +DEBUG: Plan is router executable +-- we cannot push this down since it is NOT IN +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id NOT IN (SELECT raw_events_second.user_id + FROM raw_events_second, raw_events_first + WHERE raw_events_second.user_id = raw_events_first.user_id AND raw_events_first.user_id = 200); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- safe to push down +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE EXISTS (SELECT 1 + FROM raw_events_second + WHERE raw_events_second.user_id =raw_events_first.user_id); +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300000 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300004 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300005 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300006 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((EXISTS (SELECT 1 FROM public.raw_events_second_13300007 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable +-- we cannot push down +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE NOT EXISTS (SELECT 1 + FROM raw_events_second + WHERE raw_events_second.user_id =raw_events_first.user_id); +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300000 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300004 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300005 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300006 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((NOT (EXISTS (SELECT 1 FROM public.raw_events_second_13300007 raw_events_second WHERE (raw_events_second.user_id OPERATOR(pg_catalog.=) raw_events_first.user_id)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable +-- more complex LEFT JOINs + INSERT INTO agg_events + (user_id, value_4_agg) + SELECT + outer_most.id, max(outer_most.value) + FROM + ( + SELECT f2.id as id, f2.v4 as value FROM + (SELECT + id + FROM (SELECT raw_events_first.user_id AS id + FROM raw_events_first LEFT JOIN + reference_table + ON (raw_events_first.user_id = reference_table.user_id)) AS foo) as f + LEFT JOIN + (SELECT v4, + v1, + id + FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 + ON (f.id = f2.id)) as outer_most + GROUP BY + outer_most.id; +DEBUG: distributed statement: INSERT INTO public.agg_events_13300008 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300000 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300000 raw_events_first, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) GROUP BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300009 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300001 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300010 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300002 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 1073741823)) GROUP BY id +DEBUG: distributed statement: INSERT INTO public.agg_events_13300011 AS citus_table_alias (user_id, value_4_agg) SELECT id, max(value) AS max FROM (SELECT f2.id, f2.v4 AS value FROM ((SELECT foo.id FROM (SELECT raw_events_first.user_id AS id FROM (public.raw_events_first_13300003 raw_events_first LEFT JOIN public.reference_table_13300012 reference_table ON ((raw_events_first.user_id OPERATOR(pg_catalog.=) reference_table.user_id)))) foo) f LEFT JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id)))) outer_most WHERE ((worker_hash(id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY id +DEBUG: Plan is router executable +-- cannot push down since the f.id IN is matched with value_1 +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN ( +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT value_1 + FROM raw_events_second)); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- same as above, but this time is it safe to push down since +-- f.id IN is matched with user_id +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN ( +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second)); +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300004 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300000 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300000 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300000 raw_events_first_1, public.raw_events_second_13300004 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300004 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300005 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300001 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300001 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300001 raw_events_first_1, public.raw_events_second_13300005 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300005 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300006 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300002 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300002 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300002 raw_events_first_1, public.raw_events_second_13300006 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300006 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: distributed statement: INSERT INTO public.raw_events_second_13300007 AS citus_table_alias (user_id) SELECT user_id FROM public.raw_events_first_13300003 raw_events_first WHERE ((user_id OPERATOR(pg_catalog.=) ANY (SELECT f2.id FROM ((SELECT foo.id FROM (SELECT reference_table.user_id AS id FROM public.raw_events_first_13300003 raw_events_first_1, public.reference_table_13300012 reference_table WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) reference_table.user_id)) foo) f JOIN (SELECT foo2.v4, foo2.v1, foo2.id FROM (SELECT sum(raw_events_second.value_4) AS v4, sum(raw_events_first_1.value_1) AS v1, raw_events_second.user_id AS id FROM public.raw_events_first_13300003 raw_events_first_1, public.raw_events_second_13300007 raw_events_second WHERE (raw_events_first_1.user_id OPERATOR(pg_catalog.=) raw_events_second.user_id) GROUP BY raw_events_second.user_id HAVING (sum(raw_events_second.value_4) OPERATOR(pg_catalog.>) (10)::numeric)) foo2) f2 ON ((f.id OPERATOR(pg_catalog.=) f2.id))) WHERE (f.id OPERATOR(pg_catalog.=) ANY (SELECT raw_events_second.user_id FROM public.raw_events_second_13300007 raw_events_second)))) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647))) +DEBUG: Plan is router executable +-- cannot push down since top level user_id is matched with NOT IN +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id NOT IN ( +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id = f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second)); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- cannot push down since join is not equi join (f.id > f2.id) +INSERT INTO raw_events_second + (user_id) +SELECT user_id +FROM raw_events_first +WHERE user_id IN ( +SELECT f2.id FROM +(SELECT + id +FROM (SELECT reference_table.user_id AS id + FROM raw_events_first, + reference_table + WHERE raw_events_first.user_id = reference_table.user_id ) AS foo) as f +INNER JOIN +(SELECT v4, + v1, + id +FROM (SELECT SUM(raw_events_second.value_4) AS v4, + SUM(raw_events_first.value_1) AS v1, + raw_events_second.user_id AS id + FROM raw_events_first, + raw_events_second + WHERE raw_events_first.user_id = raw_events_second.user_id + GROUP BY raw_events_second.user_id + HAVING SUM(raw_events_second.value_4) > 10) AS foo2 ) as f2 +ON (f.id > f2.id) +WHERE f.id IN (SELECT user_id + FROM raw_events_second)); +ERROR: cannot perform distributed planning for the given modification +DETAIL: Select query cannot be pushed down to the worker. +-- we currently not support grouping sets +INSERT INTO agg_events + (user_id, + value_1_agg, + value_2_agg) +SELECT user_id, + Sum(value_1) AS sum_val1, + Sum(value_2) AS sum_val2 +FROM raw_events_second +GROUP BY grouping sets ( ( user_id ), ( value_1 ), ( user_id, value_1 ), ( ) ); +DEBUG: grouping sets are not allowed in distributed INSERT ... SELECT queries +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +ERROR: could not run distributed query with GROUPING SETS, CUBE, or ROLLUP +HINT: Consider using an equality filter on the distributed table's partition column. +-- set back to INFO +SET client_min_messages TO INFO; +-- avoid constraint violations +TRUNCATE raw_events_first; +-- we don't support LIMIT for subquery pushdown, but +-- we recursively plan the query and run it via coordinator +INSERT INTO agg_events(user_id) +SELECT user_id +FROM users_table +WHERE user_id + IN (SELECT + user_id + FROM ( + ( + SELECT + user_id + FROM + ( + SELECT + e1.user_id + FROM + users_table u1, events_table e1 + WHERE + e1.user_id = u1.user_id LIMIT 3 + ) as f_inner + ) + ) AS f2); +-- Altering a table and selecting from it using a multi-shard statement +-- in the same transaction is allowed because we will use the same +-- connections for all co-located placements. +BEGIN; +ALTER TABLE raw_events_second DROP COLUMN value_4; +INSERT INTO raw_events_first SELECT * FROM raw_events_second; +ROLLBACK; +-- Alterating a table and selecting from it using a single-shard statement +-- in the same transaction is disallowed because we will use a different +-- connection. +BEGIN; +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 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); +ROLLBACK; +-- the same test with sequential DDL should work fine +BEGIN; +SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; +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); +ROLLBACK; +-- Insert after copy is allowed +BEGIN; +COPY raw_events_second (user_id, value_1) FROM STDIN DELIMITER ','; +INSERT INTO raw_events_first SELECT * FROM raw_events_second; +ROLLBACK; +-- Insert after copy is currently allowed for single-shard operation. +-- Both insert and copy are rolled back successfully. +BEGIN; +COPY raw_events_second (user_id, value_1) FROM STDIN DELIMITER ','; +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 101; +SELECT user_id FROM raw_events_first WHERE user_id = 101; + user_id +--------- + 101 +(1 row) + +ROLLBACK; +BEGIN; +INSERT INTO raw_events_first SELECT * FROM raw_events_second; +COPY raw_events_first (user_id, value_1) FROM STDIN DELIMITER ','; +ROLLBACK; +BEGIN; +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 100; +COPY raw_events_first (user_id, value_1) FROM STDIN DELIMITER ','; +ROLLBACK; +-- Similarly, multi-row INSERTs will take part in transactions and reuse connections... +BEGIN; +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 100; +COPY raw_events_first (user_id, value_1) FROM STDIN DELIMITER ','; +INSERT INTO raw_events_first (user_id, value_1) VALUES (105, 105), (106, 106); +ROLLBACK; +-- selecting from views works +CREATE VIEW test_view AS SELECT * FROM raw_events_first; +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (16, now(), 60, 600, 6000.1, 60000); +SELECT count(*) FROM raw_events_second; + count +------- + 36 +(1 row) + +INSERT INTO raw_events_second SELECT * FROM test_view; +INSERT INTO raw_events_first (user_id, time, value_1, value_2, value_3, value_4) VALUES + (17, now(), 60, 600, 6000.1, 60000); +INSERT INTO raw_events_second SELECT * FROM test_view WHERE user_id = 17 GROUP BY 1,2,3,4,5,6; +SELECT count(*) FROM raw_events_second; + count +------- + 38 +(1 row) + +-- intermediate results (CTEs) should be allowed when doing INSERT...SELECT within a CTE +WITH series AS ( + SELECT s AS val FROM generate_series(60,70) s +), +inserts AS ( + INSERT INTO raw_events_second (user_id) + SELECT + user_id + FROM + raw_events_first JOIN series ON (value_1 = val) + RETURNING + NULL +) +SELECT count(*) FROM inserts; + count +------- + 2 +(1 row) + +-- we need this in our next test +truncate raw_events_first; +SET client_min_messages TO DEBUG2; +-- first show that the query works now +INSERT INTO raw_events_first SELECT * FROM raw_events_second; +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300004 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300005 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300006 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300007 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +truncate raw_events_first; +SET client_min_messages TO DEBUG2; +-- now show that it works for a single shard query as well +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 5; +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300004 raw_events_second WHERE ((user_id OPERATOR(pg_catalog.=) 5) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: Skipping target shard interval 13300001 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300002 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300003 since SELECT query for it pruned away +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +-- if a single shard of the SELECT is unhealty, the query should fail +UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 13300004 AND nodeport = :worker_1_port; +truncate raw_events_first; +SET client_min_messages TO DEBUG2; +-- this should fail +INSERT INTO raw_events_first SELECT * FROM raw_events_second; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Insert query cannot be executed on all placements for shard 13300000 +-- this should also fail +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 5; +ERROR: cannot perform distributed planning for the given modification +DETAIL: Insert query cannot be executed on all placements for shard 13300000 +-- but this should work given that it hits different shard +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 6; +DEBUG: Skipping target shard interval 13300000 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300001 since SELECT query for it pruned away +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300006 raw_events_second WHERE ((user_id OPERATOR(pg_catalog.=) 6) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823))) +DEBUG: Skipping target shard interval 13300003 since SELECT query for it pruned away +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +-- mark the unhealthy placement as healthy again for the next tests +UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE shardid = 13300004 AND nodeport = :worker_1_port; +-- now that we should show that it works if one of the target shard interval is not healthy +UPDATE pg_dist_shard_placement SET shardstate = 3 WHERE shardid = 13300000 AND nodeport = :worker_1_port; +truncate raw_events_first; +SET client_min_messages TO DEBUG2; +-- this should work +INSERT INTO raw_events_first SELECT * FROM raw_events_second; +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300004 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer)) +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300005 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-1073741824'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300006 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 1073741823)) +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300007 raw_events_second WHERE ((worker_hash(user_id) OPERATOR(pg_catalog.>=) 1073741824) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +truncate raw_events_first; +SET client_min_messages TO DEBUG2; +-- this should also work +INSERT INTO raw_events_first SELECT * FROM raw_events_second WHERE user_id = 5; +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id, "time", value_1, value_2, value_3, value_4) SELECT user_id, "time", value_1, value_2, value_3, value_4 FROM public.raw_events_second_13300004 raw_events_second WHERE ((user_id OPERATOR(pg_catalog.=) 5) AND ((worker_hash(user_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(user_id) OPERATOR(pg_catalog.<=) '-1073741825'::integer))) +DEBUG: Skipping target shard interval 13300001 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300002 since SELECT query for it pruned away +DEBUG: Skipping target shard interval 13300003 since SELECT query for it pruned away +DEBUG: Plan is router executable +SET client_min_messages TO INFO; +-- now do some tests with varchars +INSERT INTO insert_select_varchar_test VALUES ('test_1', 10); +INSERT INTO insert_select_varchar_test VALUES ('test_2', 30); +INSERT INTO insert_select_varchar_test (key, value) +SELECT *, 100 +FROM (SELECT f1.key + FROM (SELECT key + FROM insert_select_varchar_test + GROUP BY 1 + HAVING Count(key) < 3) AS f1, + (SELECT key + FROM insert_select_varchar_test + GROUP BY 1 + HAVING Sum(COALESCE(insert_select_varchar_test.value, 0)) > + 20.0) + AS f2 + WHERE f1.key = f2.key + GROUP BY 1) AS foo; +SELECT * FROM insert_select_varchar_test ORDER BY 1 DESC, 2 DESC; + key | value +--------+------- + test_2 | 100 + test_2 | 30 + test_1 | 10 +(3 rows) + +-- some tests with DEFAULT columns and constant values +-- this test is mostly importantly intended for deparsing the query correctly +-- but still it is preferable to have this test here instead of multi_deparse_shard_query +CREATE TABLE table_with_defaults +( + store_id int, + first_name text, + default_1 int DEFAULT 1, + last_name text, + default_2 text DEFAULT '2' +); +-- we don't need many shards +SET citus.shard_count = 2; +SELECT create_distributed_table('table_with_defaults', 'store_id'); + create_distributed_table +-------------------------- + +(1 row) + +-- let's see the queries +SET client_min_messages TO DEBUG2; +-- a very simple query +INSERT INTO table_with_defaults SELECT * FROM table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, default_1, last_name, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, default_1, last_name, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- see that defaults are filled +INSERT INTO table_with_defaults (store_id, first_name) +SELECT + store_id, first_name +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, '2'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, '2'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- shuffle one of the defaults and skip the other +INSERT INTO table_with_defaults (default_2, store_id, first_name) +SELECT + default_2, store_id, first_name +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, 1 AS default_1, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- shuffle both defaults +INSERT INTO table_with_defaults (default_2, store_id, default_1, first_name) +SELECT + default_2, store_id, default_1, first_name +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, default_1, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, first_name, default_1, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- use constants instead of non-default column +INSERT INTO table_with_defaults (default_2, last_name, store_id, first_name) +SELECT + default_2, 'Freund', store_id, 'Andres' +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- use constants instead of non-default column and skip both defauls +INSERT INTO table_with_defaults (last_name, store_id, first_name) +SELECT + 'Freund', store_id, 'Andres' +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, '2'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, 'Freund'::text AS last_name, '2'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- use constants instead of default columns +INSERT INTO table_with_defaults (default_2, last_name, store_id, first_name, default_1) +SELECT + 20, last_name, store_id, first_name, 10 +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, 10, last_name, 20 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, first_name, 10, last_name, 20 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- use constants instead of both default columns and non-default columns +INSERT INTO table_with_defaults (default_2, last_name, store_id, first_name, default_1) +SELECT + 20, 'Freund', store_id, 'Andres', 10 +FROM + table_with_defaults; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 10, 'Freund'::text AS last_name, 20 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, last_name, default_2) SELECT store_id, 'Andres'::text AS first_name, 10, 'Freund'::text AS last_name, 20 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) +DEBUG: Plan is router executable +-- some of the the ultimate queries where we have constants, +-- defaults and group by entry is not on the target entry +INSERT INTO table_with_defaults (default_2, store_id, first_name) +SELECT + '2000', store_id, 'Andres' +FROM + table_with_defaults +GROUP BY + last_name, store_id; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, '2000'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1 AS default_1, '2000'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id +DEBUG: Plan is router executable +INSERT INTO table_with_defaults (default_1, store_id, first_name, default_2) +SELECT + 1000, store_id, 'Andres', '2000' +FROM + table_with_defaults +GROUP BY + last_name, store_id, first_name; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id, first_name +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id, first_name +DEBUG: Plan is router executable +INSERT INTO table_with_defaults (default_1, store_id, first_name, default_2) +SELECT + 1000, store_id, 'Andres', '2000' +FROM + table_with_defaults +GROUP BY + last_name, store_id, first_name, default_2; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id, first_name, default_2 +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2000'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id, first_name, default_2 +DEBUG: Plan is router executable +INSERT INTO table_with_defaults (default_1, store_id, first_name) +SELECT + 1000, store_id, 'Andres' +FROM + table_with_defaults +GROUP BY + last_name, store_id, first_name, default_2; +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300017 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2'::text AS default_2 FROM public.table_with_defaults_13300017 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) '-2147483648'::integer) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) '-1'::integer)) GROUP BY last_name, store_id, first_name, default_2 +DEBUG: distributed statement: INSERT INTO public.table_with_defaults_13300018 AS citus_table_alias (store_id, first_name, default_1, default_2) SELECT store_id, 'Andres'::text AS first_name, 1000, '2'::text AS default_2 FROM public.table_with_defaults_13300018 table_with_defaults WHERE ((worker_hash(store_id) OPERATOR(pg_catalog.>=) 0) AND (worker_hash(store_id) OPERATOR(pg_catalog.<=) 2147483647)) GROUP BY last_name, store_id, first_name, default_2 +DEBUG: Plan is router executable +RESET client_min_messages; +-- Stable function in default should be allowed +ALTER TABLE table_with_defaults ADD COLUMN t timestamptz DEFAULT now(); +INSERT INTO table_with_defaults (store_id, first_name, last_name) +SELECT + store_id, 'first '||store_id, 'last '||store_id +FROM + table_with_defaults +GROUP BY + store_id, first_name, last_name; +-- Volatile function in default should be disallowed +CREATE TABLE table_with_serial ( + store_id int, + s bigserial +); +SELECT create_distributed_table('table_with_serial', 'store_id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO table_with_serial (store_id) +SELECT + store_id +FROM + table_with_defaults +GROUP BY + store_id; +ERROR: INSERT ... SELECT cannot generate sequence values when selecting from a distributed table +-- do some more error/error message checks +SET citus.shard_count TO 4; +SET citus.shard_replication_factor TO 1; +CREATE TABLE text_table (part_col text, val int); +CREATE TABLE char_table (part_col char[], val int); +create table table_with_starts_with_defaults (a int DEFAULT 5, b int, c int); +SELECT create_distributed_table('text_table', 'part_col'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('char_table','part_col'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('table_with_starts_with_defaults', 'c'); + create_distributed_table +-------------------------- + +(1 row) + +SET client_min_messages TO DEBUG; +INSERT INTO text_table (part_col) + SELECT + CASE WHEN part_col = 'onder' THEN 'marco' + END +FROM text_table ; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains a case expression in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT COALESCE(part_col, 'onder') FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains a coalesce expression in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT GREATEST(part_col, 'jason') FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains a min/max expression in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT LEAST(part_col, 'andres') FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains a min/max expression in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT NULLIF(part_col, 'metin') FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT part_col isnull FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT part_col::text from char_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an explicit coercion in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT (part_col = 'burak') is true FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an expression that is not a simple column reference in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT val FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: The data type of the target table's partition column should exactly match the data type of the corresponding simple column reference in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +INSERT INTO text_table (part_col) SELECT val::text FROM text_table; +DEBUG: cannot perform distributed INSERT INTO ... SELECT because the partition columns in the source table and subquery do not match +DETAIL: Subquery contains an explicit coercion in the same position as the target table's partition column. +HINT: Ensure the target table's partition column has a corresponding simple column reference to a distributed table's partition column in the subquery. +DEBUG: Collecting INSERT ... SELECT results on coordinator +DEBUG: Router planner cannot handle multi-shard select queries +RESET client_min_messages; +insert into table_with_starts_with_defaults (b,c) select b,c FROM table_with_starts_with_defaults; +-- Test on partition column without native hash function +CREATE TABLE raw_table +( + id BIGINT, + time DATE +); +CREATE TABLE summary_table +( + time DATE, + count BIGINT +); +SELECT create_distributed_table('raw_table', 'time'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('summary_table', 'time'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO raw_table VALUES(1, '11-11-1980'); +INSERT INTO summary_table SELECT time, COUNT(*) FROM raw_table GROUP BY time; +SELECT * FROM summary_table; + time | count +------------+------- + 11-11-1980 | 1 +(1 row) + +-- Test INSERT ... SELECT via coordinator +-- Select from constants +TRUNCATE raw_events_first; +INSERT INTO raw_events_first (user_id, value_1) +SELECT * FROM (VALUES (1,2), (3,4), (5,6)) AS v(int,int); +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id; + user_id | value_1 +---------+--------- + 1 | 2 + 3 | 4 + 5 | 6 +(3 rows) + +-- Select from local functions +TRUNCATE raw_events_first; +CREATE SEQUENCE insert_select_test_seq; +SET client_min_messages TO DEBUG; +INSERT INTO raw_events_first (user_id, value_1, value_2) +SELECT + s, nextval('insert_select_test_seq'), (random()*10)::int +FROM + generate_series(1, 5) s; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: Collecting INSERT ... SELECT results on coordinator +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; +DEBUG: Router planner cannot handle multi-shard select queries + user_id | value_1 +---------+--------- + 1 | 1 + 2 | 2 + 3 | 3 + 4 | 4 + 5 | 5 +(5 rows) + +-- ON CONFLICT is supported +INSERT INTO raw_events_first (user_id, value_1) +SELECT s, nextval('insert_select_test_seq') FROM generate_series(1, 5) s +ON CONFLICT DO NOTHING; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_206_13300000'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) ON CONFLICT DO NOTHING +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_206_13300001'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) ON CONFLICT DO NOTHING +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_206_13300002'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) ON CONFLICT DO NOTHING +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_206_13300003'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) ON CONFLICT DO NOTHING +DEBUG: Collecting INSERT ... SELECT results on coordinator +-- RETURNING is supported +INSERT INTO raw_events_first (user_id, value_1) +SELECT s, nextval('insert_select_test_seq') FROM generate_series(1, 5) s +RETURNING *; +DEBUG: distributed INSERT ... SELECT can only select from distributed tables +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300000 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_207_13300000'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300001 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_207_13300001'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300002 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_207_13300002'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: distributed statement: INSERT INTO public.raw_events_first_13300003 AS citus_table_alias (user_id, value_1) SELECT user_id, value_1 FROM read_intermediate_result('insert_select_207_13300003'::text, 'binary'::citus_copy_format) intermediate_result(user_id integer, value_1 integer) RETURNING citus_table_alias.user_id, citus_table_alias."time", citus_table_alias.value_1, citus_table_alias.value_2, citus_table_alias.value_3, citus_table_alias.value_4 +DEBUG: Collecting INSERT ... SELECT results on coordinator + user_id | time | value_1 | value_2 | value_3 | value_4 +---------+------+---------+---------+---------+--------- + 1 | | 11 | | | + 2 | | 12 | | | + 3 | | 13 | | | + 4 | | 14 | | | + 5 | | 15 | | | +(5 rows) + +RESET client_min_messages; +-- INSERT ... SELECT and multi-shard SELECT in the same transaction is supported +TRUNCATE raw_events_first; +BEGIN; +INSERT INTO raw_events_first (user_id, value_1) +SELECT s, s FROM generate_series(1, 5) s; +SELECT user_id, value_1 FROM raw_events_first ORDER BY 1; + user_id | value_1 +---------+--------- + 1 | 1 + 2 | 2 + 3 | 3 + 4 | 4 + 5 | 5 +(5 rows) + +ROLLBACK; +-- INSERT ... SELECT and single-shard SELECT in the same transaction is supported +TRUNCATE raw_events_first; +BEGIN; +INSERT INTO raw_events_first (user_id, value_1) +SELECT s, s FROM generate_series(1, 5) s; +SELECT user_id, value_1 FROM raw_events_first WHERE user_id = 1; + user_id | value_1 +---------+--------- + 1 | 1 +(1 row) + +COMMIT; +-- Select from local table +TRUNCATE raw_events_first; +CREATE TEMPORARY TABLE raw_events_first_local AS +SELECT s AS u, 2*s AS v FROM generate_series(1, 5) s; +INSERT INTO raw_events_first (user_id, value_1) +SELECT u, v FROM raw_events_first_local; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 1 | 2 + 2 | 4 + 3 | 6 + 4 | 8 + 5 | 10 +(5 rows) + +-- Use columns in opposite order +TRUNCATE raw_events_first; +INSERT INTO raw_events_first (value_1, user_id) +SELECT u, v FROM raw_events_first_local; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 2 | 1 + 4 | 2 + 6 | 3 + 8 | 4 + 10 | 5 +(5 rows) + +-- Set operations can work with opposite column order +TRUNCATE raw_events_first; +INSERT INTO raw_events_first (value_3, user_id) +( SELECT v, u::bigint FROM raw_events_first_local ) +UNION ALL +( SELECT v, u FROM raw_events_first_local ); +SELECT user_id, value_3 FROM raw_events_first ORDER BY user_id, value_3; + user_id | value_3 +---------+--------- + 1 | 2 + 1 | 2 + 2 | 4 + 2 | 4 + 3 | 6 + 3 | 6 + 4 | 8 + 4 | 8 + 5 | 10 + 5 | 10 +(10 rows) + +-- Select from other distributed table with limit +TRUNCATE raw_events_first; +TRUNCATE raw_events_second; +INSERT INTO raw_events_second (user_id, value_4) +SELECT s, 3*s FROM generate_series (1,5) s; +INSERT INTO raw_events_first (user_id, value_1) +SELECT user_id, value_4 FROM raw_events_second LIMIT 5; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 1 | 3 + 2 | 6 + 3 | 9 + 4 | 12 + 5 | 15 +(5 rows) + +-- CTEs are supported in local queries +TRUNCATE raw_events_first; +WITH removed_rows AS ( + DELETE FROM raw_events_first_local RETURNING u +) +INSERT INTO raw_events_first (user_id, value_1) +WITH value AS (SELECT 1) +SELECT * FROM removed_rows, value; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 1 | 1 + 2 | 1 + 3 | 1 + 4 | 1 + 5 | 1 +(5 rows) + +-- nested CTEs are also supported +TRUNCATE raw_events_first; +INSERT INTO raw_events_first_local SELECT s, 2*s FROM generate_series(0, 10) s; +WITH rows_to_remove AS ( + SELECT u FROM raw_events_first_local WHERE u > 0 +), +removed_rows AS ( + DELETE FROM raw_events_first_local + WHERE u IN (SELECT * FROM rows_to_remove) + RETURNING u, v +) +INSERT INTO raw_events_first (user_id, value_1) +WITH ultra_rows AS ( + WITH numbers AS ( + SELECT s FROM generate_series(1,10) s + ), + super_rows AS ( + SELECT u, v FROM removed_rows JOIN numbers ON (u = s) + ) + SELECT * FROM super_rows LIMIT 5 +) +SELECT u, v FROM ultra_rows; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 1 | 2 + 2 | 4 + 3 | 6 + 4 | 8 + 5 | 10 +(5 rows) + +-- CTEs with duplicate names are also supported +TRUNCATE raw_events_first; +WITH super_rows AS ( + SELECT u FROM raw_events_first_local +) +INSERT INTO raw_events_first (user_id, value_1) +WITH super_rows AS ( + SELECT * FROM super_rows GROUP BY u +) +SELECT u, 5 FROM super_rows; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 0 | 5 +(1 row) + +-- CTEs are supported in router queries +TRUNCATE raw_events_first; +WITH user_two AS ( + SELECT user_id, value_4 FROM raw_events_second WHERE user_id = 2 +) +INSERT INTO raw_events_first (user_id, value_1) +SELECT * FROM user_two; +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 2 | 6 +(1 row) + +-- CTEs are supported when there are name collisions +WITH numbers AS ( + SELECT s FROM generate_series(1,10) s +) +INSERT INTO raw_events_first(user_id, value_1) +WITH numbers AS ( + SELECT s, s FROM generate_series(1,5) s +) +SELECT * FROM numbers; +-- Select into distributed table with a sequence +CREATE TABLE "CaseSensitiveTable" ("UserID" int, "Value1" int); +SELECT create_distributed_table('"CaseSensitiveTable"', 'UserID'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO "CaseSensitiveTable" +SELECT s, s FROM generate_series(1,10) s; +SELECT * FROM "CaseSensitiveTable" ORDER BY "UserID"; + UserID | Value1 +--------+-------- + 1 | 1 + 2 | 2 + 3 | 3 + 4 | 4 + 5 | 5 + 6 | 6 + 7 | 7 + 8 | 8 + 9 | 9 + 10 | 10 +(10 rows) + +DROP TABLE "CaseSensitiveTable"; +-- Select into distributed table with a sequence +CREATE TABLE dist_table_with_sequence (user_id serial, value_1 serial); +SELECT create_distributed_table('dist_table_with_sequence', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +-- from local query +INSERT INTO dist_table_with_sequence (value_1) +SELECT s FROM generate_series(1,5) s; +SELECT * FROM dist_table_with_sequence ORDER BY user_id; + user_id | value_1 +---------+--------- + 1 | 1 + 2 | 2 + 3 | 3 + 4 | 4 + 5 | 5 +(5 rows) + +-- from a distributed query +INSERT INTO dist_table_with_sequence (value_1) +SELECT value_1 FROM dist_table_with_sequence; +ERROR: INSERT ... SELECT cannot generate sequence values when selecting from a distributed table +SELECT * FROM dist_table_with_sequence ORDER BY user_id; + user_id | value_1 +---------+--------- + 1 | 1 + 2 | 2 + 3 | 3 + 4 | 4 + 5 | 5 +(5 rows) + +-- Select from distributed table into reference table +CREATE TABLE ref_table (user_id int, value_1 int); +SELECT create_reference_table('ref_table'); + create_reference_table +------------------------ + +(1 row) + +INSERT INTO ref_table +SELECT user_id, value_1 FROM raw_events_second; +SELECT * FROM ref_table ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 1 | + 2 | + 3 | + 4 | + 5 | +(5 rows) + +DROP TABLE ref_table; +-- Select from reference table into reference table +CREATE TABLE ref1 (d timestamptz); +SELECT create_reference_table('ref1'); + create_reference_table +------------------------ + +(1 row) + +CREATE TABLE ref2 (d date); +SELECT create_reference_table('ref2'); + create_reference_table +------------------------ + +(1 row) + +INSERT INTO ref2 VALUES ('2017-10-31'); +INSERT INTO ref1 SELECT * FROM ref2; +SELECT count(*) from ref1; + count +------- + 1 +(1 row) + +-- also test with now() +INSERT INTO ref1 SELECT now() FROM ref2; +SELECT count(*) from ref1; + count +------- + 2 +(1 row) + +DROP TABLE ref1; +DROP TABLE ref2; +-- Select into an append-partitioned table is not supported +CREATE TABLE insert_append_table (user_id int, value_4 bigint); +SELECT create_distributed_table('insert_append_table', 'user_id', 'append'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO insert_append_table (user_id, value_4) +SELECT user_id, 1 FROM raw_events_second LIMIT 5; +ERROR: INSERT ... SELECT into an append-distributed table is not supported +DROP TABLE insert_append_table; +-- Insert from other distributed table as prepared statement +TRUNCATE raw_events_first; +PREPARE insert_prep(int) AS +INSERT INTO raw_events_first (user_id, value_1) +SELECT $1, value_4 FROM raw_events_second ORDER BY value_4 LIMIT 1; +EXECUTE insert_prep(1); +EXECUTE insert_prep(2); +EXECUTE insert_prep(3); +EXECUTE insert_prep(4); +EXECUTE insert_prep(5); +EXECUTE insert_prep(6); +SELECT user_id, value_1 FROM raw_events_first ORDER BY user_id, value_1; + user_id | value_1 +---------+--------- + 1 | 3 + 2 | 3 + 3 | 3 + 4 | 3 + 5 | 3 + 6 | 3 +(6 rows) + +-- Inserting into views is handled via coordinator +TRUNCATE raw_events_first; +INSERT INTO test_view +SELECT * FROM raw_events_second; +SELECT user_id, value_4 FROM test_view ORDER BY user_id, value_4; + user_id | value_4 +---------+--------- + 1 | 3 + 2 | 6 + 3 | 9 + 4 | 12 + 5 | 15 +(5 rows) + +-- Drop the view now, because the column we are about to drop depends on it +DROP VIEW test_view; +-- Make sure we handle dropped columns correctly +CREATE TABLE drop_col_table (col1 text, col2 text, col3 text); +SELECT create_distributed_table('drop_col_table', 'col2'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE drop_col_table DROP COLUMN col1; +INSERT INTO drop_col_table (col3, col2) +SELECT value_4, user_id FROM raw_events_second LIMIT 5; +SELECT * FROM drop_col_table ORDER BY col2, col3; + col2 | col3 +------+------ + 1 | 3 + 2 | 6 + 3 | 9 + 4 | 12 + 5 | 15 +(5 rows) + +-- make sure the tuple went to the right shard +SELECT * FROM drop_col_table WHERE col2 = '1'; + col2 | col3 +------+------ + 1 | 3 +(1 row) + +RESET client_min_messages; +-- make sure casts are handled correctly +CREATE TABLE coerce_events(user_id int, time timestamp, value_1 numeric); +SELECT create_distributed_table('coerce_events', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE coerce_agg (user_id int, value_1_agg int); +SELECT create_distributed_table('coerce_agg', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO coerce_events(user_id, value_1) VALUES (1, 1), (2, 2), (10, 10); +-- numeric -> int (straight function) +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop +ORDER BY 2 DESC, 1 DESC +LIMIT 5; +-- int -> text +ALTER TABLE coerce_agg ALTER COLUMN value_1_agg TYPE text; +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop +LIMIT 5; +SELECT * FROM coerce_agg ORDER BY 1 DESC, 2 DESC; + user_id | value_1_agg +---------+------------- + 10 | 10 + 10 | 10 + 2 | 2 + 2 | 2 + 1 | 1 + 1 | 1 +(6 rows) + +TRUNCATE coerce_agg; +-- int -> char(1) +ALTER TABLE coerce_agg ALTER COLUMN value_1_agg TYPE char(1); +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop +LIMIT 5; +ERROR: value too long for type character(1) +SELECT * FROM coerce_agg ORDER BY 1 DESC, 2 DESC; + user_id | value_1_agg +---------+------------- +(0 rows) + +TRUNCATE coerce_agg; +TRUNCATE coerce_events; +-- char(5) -> char(1) +ALTER TABLE coerce_events ALTER COLUMN value_1 TYPE char(5); +INSERT INTO coerce_events(user_id, value_1) VALUES (1, 'aaaaa'), (2, 'bbbbb'); +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop +LIMIT 5; +ERROR: value too long for type character(1) +-- char(1) -> char(5) +ALTER TABLE coerce_events ALTER COLUMN value_1 TYPE char(1) USING value_1::char(1); +ALTER TABLE coerce_agg ALTER COLUMN value_1_agg TYPE char(5); +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop +LIMIT 5; +SELECT * FROM coerce_agg ORDER BY 1 DESC, 2 DESC; + user_id | value_1_agg +---------+------------- + 2 | b + 1 | a +(2 rows) + +TRUNCATE coerce_agg; +TRUNCATE coerce_events; +-- integer -> integer (check VALUE < 5) +ALTER TABLE coerce_events ALTER COLUMN value_1 TYPE integer USING NULL; +ALTER TABLE coerce_agg ALTER COLUMN value_1_agg TYPE integer USING NULL; +ALTER TABLE coerce_agg ADD CONSTRAINT small_number CHECK (value_1_agg < 5); +INSERT INTO coerce_events (user_id, value_1) VALUES (1, 1), (10, 10); +\set VERBOSITY TERSE +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop; +ERROR: new row for relation "coerce_agg_13300060" violates check constraint "small_number_13300060" +\set VERBOSITY DEFAULT +SELECT * FROM coerce_agg ORDER BY 1 DESC, 2 DESC; + user_id | value_1_agg +---------+------------- +(0 rows) + +-- integer[3] -> text[3] +TRUNCATE coerce_events; +ALTER TABLE coerce_events ALTER COLUMN value_1 TYPE integer[3] USING NULL; +INSERT INTO coerce_events(user_id, value_1) VALUES (1, '{1,1,1}'), (2, '{2,2,2}'); +ALTER TABLE coerce_agg DROP COLUMN value_1_agg; +ALTER TABLE coerce_agg ADD COLUMN value_1_agg text[3]; +INSERT INTO coerce_agg(user_id, value_1_agg) +SELECT * +FROM ( + SELECT user_id, value_1 + FROM coerce_events +) AS ftop +LIMIT 5; +SELECT * FROM coerce_agg ORDER BY 1 DESC, 2 DESC; + user_id | value_1_agg +---------+------------- + 2 | {2,2,2} + 1 | {1,1,1} +(2 rows) + +-- INSERT..SELECT + prepared statements + recursive planning +BEGIN; +PREPARE prepared_recursive_insert_select AS +INSERT INTO users_table +SELECT * FROM users_table +WHERE value_1 IN (SELECT value_2 FROM events_table OFFSET 0); +EXECUTE prepared_recursive_insert_select; +EXECUTE prepared_recursive_insert_select; +EXECUTE prepared_recursive_insert_select; +EXECUTE prepared_recursive_insert_select; +EXECUTE prepared_recursive_insert_select; +EXECUTE prepared_recursive_insert_select; +ROLLBACK; +-- upsert with on conflict update distribution column is unsupported +INSERT INTO agg_events AS ae + ( + user_id, + value_1_agg, + agg_time + ) +SELECT user_id, + value_1, + time +FROM raw_events_first +ON conflict (user_id, value_1_agg) +DO UPDATE + SET user_id = 42 +RETURNING user_id, value_1_agg; +ERROR: modifying the partition value of rows is not allowed +-- wrap in a transaction to improve performance +BEGIN; +DROP TABLE coerce_events; +DROP TABLE coerce_agg; +DROP TABLE drop_col_table; +DROP TABLE raw_table; +DROP TABLE summary_table; +DROP TABLE raw_events_first CASCADE; +DROP TABLE raw_events_second; +DROP TABLE reference_table; +DROP TABLE agg_events; +DROP TABLE table_with_defaults; +DROP TABLE table_with_serial; +DROP TABLE text_table; +DROP TABLE char_table; +DROP TABLE table_with_starts_with_defaults; +COMMIT; diff --git a/src/test/regress/expected/multi_modifications.out b/src/test/regress/expected/multi_modifications.out index 4887d55ce..01cfeaa22 100644 --- a/src/test/regress/expected/multi_modifications.out +++ b/src/test/regress/expected/multi_modifications.out @@ -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 diff --git a/src/test/regress/expected/multi_modifications_9.out b/src/test/regress/expected/multi_modifications_9.out new file mode 100644 index 000000000..7843a3d92 --- /dev/null +++ b/src/test/regress/expected/multi_modifications_9.out @@ -0,0 +1,1279 @@ +SET citus.shard_count TO 32; +SET citus.next_shard_id TO 750000; +SET citus.next_placement_id TO 750000; +-- some failure messages that comes from the worker nodes +-- might change due to parallel exectuions, so supress those +-- using \set VERBOSITY terse +-- =================================================================== +-- test end-to-end modification functionality +-- =================================================================== +CREATE TYPE order_side AS ENUM ('buy', 'sell'); +CREATE TABLE limit_orders ( + id bigint PRIMARY KEY, + symbol text NOT NULL, + bidder_id bigint NOT NULL, + placed_at timestamp NOT NULL, + kind order_side NOT NULL, + limit_price decimal NOT NULL DEFAULT 0.00 CHECK (limit_price >= 0.00) +); +CREATE TABLE multiple_hash ( + category text NOT NULL, + data text NOT NULL +); +CREATE TABLE insufficient_shards ( LIKE limit_orders ); +CREATE TABLE range_partitioned ( LIKE limit_orders ); +CREATE TABLE append_partitioned ( LIKE limit_orders ); +SET citus.shard_count TO 2; +SELECT create_distributed_table('limit_orders', 'id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('multiple_hash', 'id', 'hash'); +ERROR: column "id" of relation "multiple_hash" does not exist +SELECT create_distributed_table('range_partitioned', 'id', 'range'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('append_partitioned', 'id', 'append'); + create_distributed_table +-------------------------- + +(1 row) + +SET citus.shard_count TO 1; +SET citus.shard_replication_factor TO 1; +-- make a single shard that covers no partition values +SELECT create_distributed_table('insufficient_shards', 'id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +UPDATE pg_dist_shard SET shardminvalue = 0, shardmaxvalue = 0 +WHERE logicalrelid = 'insufficient_shards'::regclass; +-- create range-partitioned shards +SELECT master_create_empty_shard('range_partitioned') AS new_shard_id +\gset +UPDATE pg_dist_shard SET shardminvalue = 0, shardmaxvalue = 49999 +WHERE shardid = :new_shard_id; +SELECT master_create_empty_shard('range_partitioned') AS new_shard_id +\gset +UPDATE pg_dist_shard SET shardminvalue = 50000, shardmaxvalue = 99999 +WHERE shardid = :new_shard_id; +-- create append-partitioned shards +SELECT master_create_empty_shard('append_partitioned') AS new_shard_id +\gset +UPDATE pg_dist_shard SET shardminvalue = 0, shardmaxvalue = 500000 +WHERE shardid = :new_shard_id; +SELECT master_create_empty_shard('append_partitioned') AS new_shard_id +\gset +UPDATE pg_dist_shard SET shardminvalue = 500000, shardmaxvalue = 1000000 +WHERE shardid = :new_shard_id; +-- basic single-row INSERT +INSERT INTO limit_orders VALUES (32743, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', + 20.69); +SELECT COUNT(*) FROM limit_orders WHERE id = 32743; + count +------- + 1 +(1 row) + +-- basic single-row INSERT with RETURNING +INSERT INTO limit_orders VALUES (32744, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', 20.69) RETURNING *; + id | symbol | bidder_id | placed_at | kind | limit_price +-------+--------+-----------+--------------------------+------+------------- + 32744 | AAPL | 9580 | Tue Oct 19 10:23:54 2004 | buy | 20.69 +(1 row) + +-- try a single-row INSERT with no shard to receive it +INSERT INTO insufficient_shards VALUES (32743, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', + 20.69); +ERROR: cannot find shard interval +DETAIL: Hash of the partition column value does not fall into any shards. +-- try an insert to a range-partitioned table +INSERT INTO range_partitioned VALUES (32743, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', + 20.69); +-- also insert to an append-partitioned table +INSERT INTO append_partitioned VALUES (414123, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', + 20.69); +-- ensure the values are where we put them and query to ensure they are properly pruned +SET client_min_messages TO 'DEBUG2'; +RESET citus.task_executor_type; +SELECT * FROM range_partitioned WHERE id = 32743; +DEBUG: Creating router plan +DEBUG: Plan is router executable + id | symbol | bidder_id | placed_at | kind | limit_price +-------+--------+-----------+--------------------------+------+------------- + 32743 | AAPL | 9580 | Tue Oct 19 10:23:54 2004 | buy | 20.69 +(1 row) + +SELECT * FROM append_partitioned WHERE id = 414123; +DEBUG: Router planner does not support append-partitioned tables. +DEBUG: Plan is router executable + id | symbol | bidder_id | placed_at | kind | limit_price +--------+--------+-----------+--------------------------+------+------------- + 414123 | AAPL | 9580 | Tue Oct 19 10:23:54 2004 | buy | 20.69 +(1 row) + +SET client_min_messages TO DEFAULT; +SET citus.task_executor_type TO DEFAULT; +-- try inserting without a range-partitioned shard to receive the value +INSERT INTO range_partitioned VALUES (999999, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', + 20.69); +ERROR: cannot run INSERT command which targets no shards +HINT: Make sure you have created a shard which can receive this partition column value. +-- and insert into an append-partitioned table with a value that spans shards: +INSERT INTO append_partitioned VALUES (500000, 'AAPL', 9580, '2004-10-19 10:23:54', 'buy', + 20.69); +ERROR: cannot run INSERT command which targets multiple shards +HINT: Make sure the value for partition column "id" falls into a single shard. +-- INSERT with DEFAULT in the target list +INSERT INTO limit_orders VALUES (12756, 'MSFT', 10959, '2013-05-08 07:29:23', 'sell', + DEFAULT); +SELECT COUNT(*) FROM limit_orders WHERE id = 12756; + count +------- + 1 +(1 row) + +-- INSERT with expressions in target list +INSERT INTO limit_orders VALUES (430, upper('ibm'), 214, timestamp '2003-01-28 10:31:17' + + interval '5 hours', 'buy', sqrt(2)); +SELECT COUNT(*) FROM limit_orders WHERE id = 430; + count +------- + 1 +(1 row) + +-- INSERT without partition key +INSERT INTO limit_orders DEFAULT VALUES; +ERROR: cannot perform an INSERT without a partition column value +-- squelch WARNINGs that contain worker_port +SET client_min_messages TO ERROR; +-- INSERT violating NOT NULL constraint +INSERT INTO limit_orders VALUES (NULL, 'T', 975234, DEFAULT); +ERROR: cannot perform an INSERT with NULL in the partition column +-- INSERT violating column constraint +\set VERBOSITY terse +INSERT INTO limit_orders VALUES (18811, 'BUD', 14962, '2014-04-05 08:32:16', 'sell', + -5.00); +ERROR: new row for relation "limit_orders_750000" violates check constraint "limit_orders_limit_price_check" +-- INSERT violating primary key constraint +INSERT INTO limit_orders VALUES (32743, 'LUV', 5994, '2001-04-16 03:37:28', 'buy', 0.58); +ERROR: duplicate key value violates unique constraint "limit_orders_pkey_750001" +-- INSERT violating primary key constraint, with RETURNING specified. +INSERT INTO limit_orders VALUES (32743, 'LUV', 5994, '2001-04-16 03:37:28', 'buy', 0.58) RETURNING *; +ERROR: duplicate key value violates unique constraint "limit_orders_pkey_750001" +-- INSERT, with RETURNING specified, failing with a non-constraint error +INSERT INTO limit_orders VALUES (34153, 'LEE', 5994, '2001-04-16 03:37:28', 'buy', 0.58) RETURNING id / 0; +ERROR: division by zero +\set VERBOSITY DEFAULT +SET client_min_messages TO DEFAULT; +-- commands with non-constant partition values are supported +INSERT INTO limit_orders VALUES (random() * 100, 'ORCL', 152, '2011-08-25 11:50:45', + 'sell', 0.58); +-- values for other columns are totally fine +INSERT INTO limit_orders VALUES (2036, 'GOOG', 5634, now(), 'buy', random()); +-- commands with mutable functions in their quals +DELETE FROM limit_orders WHERE id = 246 AND bidder_id = (random() * 1000); +ERROR: functions used in the WHERE clause of modification queries on distributed tables must not be VOLATILE +-- commands with mutable but non-volatile functions(ie: stable func.) in their quals +-- (the cast to timestamp is because the timestamp_eq_timestamptz operator is stable) +DELETE FROM limit_orders WHERE id = 246 AND placed_at = current_timestamp::timestamp; +-- multi-row inserts are supported +INSERT INTO limit_orders VALUES (12037, 'GOOG', 5634, '2001-04-16 03:37:28', 'buy', 0.50), + (12038, 'GOOG', 5634, '2001-04-17 03:37:28', 'buy', 2.50), + (12039, 'GOOG', 5634, '2001-04-18 03:37:28', 'buy', 1.50); +SELECT COUNT(*) FROM limit_orders WHERE id BETWEEN 12037 AND 12039; + count +------- + 3 +(1 row) + +-- even those with functions and returning +INSERT INTO limit_orders VALUES (22037, 'GOOG', 5634, now(), 'buy', 0.50), + (22038, 'GOOG', 5634, now(), 'buy', 2.50), + (22039, 'GOOG', 5634, now(), 'buy', 1.50) +RETURNING id; + id +------- + 22037 + 22038 + 22039 +(3 rows) + +SELECT COUNT(*) FROM limit_orders WHERE id BETWEEN 22037 AND 22039; + count +------- + 3 +(1 row) + +-- even those with functions in their partition columns +INSERT INTO limit_orders VALUES (random() * 10 + 70000, 'GOOG', 5634, now(), 'buy', 0.50), + (random() * 10 + 80000, 'GOOG', 5634, now(), 'buy', 2.50), + (random() * 10 + 80090, 'GOOG', 5634, now(), 'buy', 1.50); +SELECT COUNT(*) FROM limit_orders WHERE id BETWEEN 70000 AND 90000; + count +------- + 3 +(1 row) + +-- commands containing a CTE are supported +WITH deleted_orders AS (DELETE FROM limit_orders WHERE id < 0 RETURNING *) +INSERT INTO limit_orders SELECT * FROM deleted_orders; +-- test simple DELETE +INSERT INTO limit_orders VALUES (246, 'TSLA', 162, '2007-07-02 16:32:15', 'sell', 20.69); +SELECT COUNT(*) FROM limit_orders WHERE id = 246; + count +------- + 1 +(1 row) + +DELETE FROM limit_orders WHERE id = 246; +SELECT COUNT(*) FROM limit_orders WHERE id = 246; + count +------- + 0 +(1 row) + +-- test simple DELETE with RETURNING +DELETE FROM limit_orders WHERE id = 430 RETURNING *; + id | symbol | bidder_id | placed_at | kind | limit_price +-----+--------+-----------+--------------------------+------+----------------- + 430 | IBM | 214 | Tue Jan 28 15:31:17 2003 | buy | 1.4142135623731 +(1 row) + +SELECT COUNT(*) FROM limit_orders WHERE id = 430; + count +------- + 0 +(1 row) + +-- DELETE with expression in WHERE clause +INSERT INTO limit_orders VALUES (246, 'TSLA', 162, '2007-07-02 16:32:15', 'sell', 20.69); +SELECT COUNT(*) FROM limit_orders WHERE id = 246; + count +------- + 1 +(1 row) + +DELETE FROM limit_orders WHERE id = (2 * 123); +SELECT COUNT(*) FROM limit_orders WHERE id = 246; + count +------- + 0 +(1 row) + +-- commands with a USING clause are supported +CREATE TABLE bidders ( name text, id bigint ); +DELETE FROM limit_orders USING bidders WHERE limit_orders.id = 246 AND + limit_orders.bidder_id = bidders.id AND + bidders.name = 'Bernie Madoff'; +ERROR: relation bidders is not distributed +-- commands containing a CTE are supported +WITH new_orders AS (INSERT INTO limit_orders VALUES (411, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66)) +DELETE FROM limit_orders WHERE id < 0; +-- we have to be careful that modifying CTEs are part of the transaction and can thus roll back +\set VERBOSITY terse +WITH new_orders AS (INSERT INTO limit_orders VALUES (412, 'FLO', 12, '2017-07-02 16:32:15', 'buy', 66)) +DELETE FROM limit_orders RETURNING id / 0; +ERROR: division by zero +\set VERBOSITY default +SELECT * FROM limit_orders WHERE id = 412; + id | symbol | bidder_id | placed_at | kind | limit_price +----+--------+-----------+-----------+------+------------- +(0 rows) + +INSERT INTO limit_orders VALUES (246, 'TSLA', 162, '2007-07-02 16:32:15', 'sell', 20.69); +-- simple UPDATE +UPDATE limit_orders SET symbol = 'GM' WHERE id = 246; +SELECT symbol FROM limit_orders WHERE id = 246; + symbol +-------- + GM +(1 row) + +-- simple UPDATE with RETURNING +UPDATE limit_orders SET symbol = 'GM' WHERE id = 246 RETURNING *; + id | symbol | bidder_id | placed_at | kind | limit_price +-----+--------+-----------+--------------------------+------+------------- + 246 | GM | 162 | Mon Jul 02 16:32:15 2007 | sell | 20.69 +(1 row) + +-- expression UPDATE +UPDATE limit_orders SET bidder_id = 6 * 3 WHERE id = 246; +SELECT bidder_id FROM limit_orders WHERE id = 246; + bidder_id +----------- + 18 +(1 row) + +-- expression UPDATE with RETURNING +UPDATE limit_orders SET bidder_id = 6 * 5 WHERE id = 246 RETURNING *; + id | symbol | bidder_id | placed_at | kind | limit_price +-----+--------+-----------+--------------------------+------+------------- + 246 | GM | 30 | Mon Jul 02 16:32:15 2007 | sell | 20.69 +(1 row) + +-- multi-column UPDATE +UPDATE limit_orders SET (kind, limit_price) = ('buy', DEFAULT) WHERE id = 246; +SELECT kind, limit_price FROM limit_orders WHERE id = 246; + kind | limit_price +------+------------- + buy | 0.00 +(1 row) + +-- multi-column UPDATE with RETURNING +UPDATE limit_orders SET (kind, limit_price) = ('buy', 999) WHERE id = 246 RETURNING *; + id | symbol | bidder_id | placed_at | kind | limit_price +-----+--------+-----------+--------------------------+------+------------- + 246 | GM | 30 | Mon Jul 02 16:32:15 2007 | buy | 999 +(1 row) + +-- Test that on unique contraint violations, we fail fast +\set VERBOSITY terse +INSERT INTO limit_orders VALUES (275, 'ADR', 140, '2007-07-02 16:32:15', 'sell', 43.67); +INSERT INTO limit_orders VALUES (275, 'ADR', 140, '2007-07-02 16:32:15', 'sell', 43.67); +ERROR: duplicate key value violates unique constraint "limit_orders_pkey_750001" +-- Test that shards which miss a modification are marked unhealthy +-- First: Connect to the second worker node +\c - - - :worker_2_port +-- Second: Move aside limit_orders shard on the second worker node +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); +ERROR: relation "public.limit_orders_750000" does not exist +-- 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 +------- + 0 +(1 row) + +SELECT count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND sp.shardstate = 3 +AND s.logicalrelid = 'limit_orders'::regclass; + count +------- + 0 +(1 row) + +-- Test that if all shards miss a modification, no state change occurs +-- First: Connect to the first worker node +\c - - - :worker_1_port +-- Second: Move aside limit_orders shard on the second worker node +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 +\set VERBOSITY terse +INSERT INTO limit_orders VALUES (276, 'ADR', 140, '2007-07-02 16:32:15', 'sell', 43.67); +ERROR: relation "public.limit_orders_750000" does not exist +\set VERBOSITY DEFAULT +-- Last: Verify worker is still healthy +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_1_port +AND sp.shardstate = 1 +AND s.logicalrelid = 'limit_orders'::regclass; + count +------- + 2 +(1 row) + +-- Undo our change... +-- First: Connect to the first worker node +\c - - - :worker_1_port +-- Second: Move aside limit_orders shard on the second worker node +ALTER TABLE renamed_orders RENAME TO limit_orders_750000; +-- Third: Connect back to master node +\c - - - :master_port +-- attempting to change the partition key is unsupported +UPDATE limit_orders SET id = 0 WHERE id = 246; +ERROR: modifying the partition value of rows is not allowed +UPDATE limit_orders SET id = 0 WHERE id = 0 OR id = 246; +ERROR: modifying the partition value of rows is not allowed +-- setting the partition column value to itself is allowed +UPDATE limit_orders SET id = 246 WHERE id = 246; +UPDATE limit_orders SET id = 246 WHERE id = 246 AND symbol = 'GM'; +UPDATE limit_orders SET id = limit_orders.id WHERE id = 246; +-- UPDATEs with a FROM clause are unsupported +UPDATE limit_orders SET limit_price = 0.00 FROM bidders + WHERE limit_orders.id = 246 AND + limit_orders.bidder_id = bidders.id AND + bidders.name = 'Bernie Madoff'; +ERROR: relation bidders is not distributed +-- should succeed with a CTE +WITH deleted_orders AS (INSERT INTO limit_orders VALUES (399, 'PDR', 14, '2017-07-02 16:32:15', 'sell', 43)) +UPDATE limit_orders SET symbol = 'GM'; +SELECT symbol, bidder_id FROM limit_orders WHERE id = 246; + symbol | bidder_id +--------+----------- + GM | 30 +(1 row) + +-- updates referencing just a var are supported +UPDATE limit_orders SET bidder_id = id WHERE id = 246; +-- updates referencing a column are supported +UPDATE limit_orders SET bidder_id = bidder_id + 1 WHERE id = 246; +-- IMMUTABLE functions are allowed +UPDATE limit_orders SET symbol = LOWER(symbol) WHERE id = 246; +SELECT symbol, bidder_id FROM limit_orders WHERE id = 246; + symbol | bidder_id +--------+----------- + gm | 247 +(1 row) + +-- IMMUTABLE functions are allowed -- even in returning +UPDATE limit_orders SET symbol = UPPER(symbol) WHERE id = 246 RETURNING id, LOWER(symbol), symbol; + id | lower | symbol +-----+-------+-------- + 246 | gm | GM +(1 row) + +ALTER TABLE limit_orders ADD COLUMN array_of_values integer[]; +-- updates referencing STABLE functions are allowed +UPDATE limit_orders SET placed_at = LEAST(placed_at, now()::timestamp) WHERE id = 246; +-- so are binary operators +UPDATE limit_orders SET array_of_values = 1 || array_of_values WHERE id = 246; +CREATE FUNCTION immutable_append(old_values int[], new_value int) +RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; +\c - - - :worker_1_port +CREATE FUNCTION immutable_append(old_values int[], new_value int) +RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; +\c - - - :worker_2_port +CREATE FUNCTION immutable_append(old_values int[], new_value int) +RETURNS int[] AS $$ SELECT old_values || new_value $$ LANGUAGE SQL IMMUTABLE; +\c - - - :master_port +-- immutable function calls with vars are also allowed +UPDATE limit_orders +SET array_of_values = immutable_append(array_of_values, 2) WHERE id = 246; +CREATE FUNCTION stable_append(old_values int[], new_value int) +RETURNS int[] AS $$ BEGIN RETURN old_values || new_value; END; $$ +LANGUAGE plpgsql STABLE; +-- but STABLE function calls with vars are not allowed +UPDATE limit_orders +SET array_of_values = stable_append(array_of_values, 3) WHERE id = 246; +ERROR: STABLE functions used in UPDATE queries cannot be called with column references +SELECT array_of_values FROM limit_orders WHERE id = 246; + array_of_values +----------------- + {1,2} +(1 row) + +-- STRICT functions work as expected +CREATE FUNCTION temp_strict_func(integer,integer) RETURNS integer AS +'SELECT COALESCE($1, 2) + COALESCE($1, 3);' LANGUAGE SQL STABLE STRICT; +\set VERBOSITY terse +UPDATE limit_orders SET bidder_id = temp_strict_func(1, null) WHERE id = 246; +ERROR: null value in column "bidder_id" violates not-null constraint +\set VERBOSITY default +SELECT array_of_values FROM limit_orders WHERE id = 246; + array_of_values +----------------- + {1,2} +(1 row) + +ALTER TABLE limit_orders DROP array_of_values; +-- even in RETURNING +UPDATE limit_orders SET placed_at = placed_at WHERE id = 246 RETURNING NOW(); +ERROR: non-IMMUTABLE functions are not allowed in the RETURNING clause +-- check that multi-row UPDATE/DELETEs with RETURNING work +INSERT INTO multiple_hash VALUES ('0', '1'); +INSERT INTO multiple_hash VALUES ('0', '2'); +INSERT INTO multiple_hash VALUES ('0', '3'); +INSERT INTO multiple_hash VALUES ('0', '4'); +INSERT INTO multiple_hash VALUES ('0', '5'); +INSERT INTO multiple_hash VALUES ('0', '6'); +UPDATE multiple_hash SET data = data ||'-1' WHERE category = '0' RETURNING *; + category | data +----------+------ + 0 | 1-1 + 0 | 2-1 + 0 | 3-1 + 0 | 4-1 + 0 | 5-1 + 0 | 6-1 +(6 rows) + +DELETE FROM multiple_hash WHERE category = '0' RETURNING *; + category | data +----------+------ + 0 | 1-1 + 0 | 2-1 + 0 | 3-1 + 0 | 4-1 + 0 | 5-1 + 0 | 6-1 +(6 rows) + +-- ensure returned row counters are correct +\set QUIET off +INSERT INTO multiple_hash VALUES ('1', '1'); +INSERT 0 1 +INSERT INTO multiple_hash VALUES ('1', '2'); +INSERT 0 1 +INSERT INTO multiple_hash VALUES ('1', '3'); +INSERT 0 1 +INSERT INTO multiple_hash VALUES ('2', '1'); +INSERT 0 1 +INSERT INTO multiple_hash VALUES ('2', '2'); +INSERT 0 1 +INSERT INTO multiple_hash VALUES ('2', '3'); +INSERT 0 1 +INSERT INTO multiple_hash VALUES ('2', '3') RETURNING *; + category | data +----------+------ + 2 | 3 +(1 row) + +INSERT 0 1 +-- check that update return the right number of rows +-- one row +UPDATE multiple_hash SET data = data ||'-1' WHERE category = '1' AND data = '1'; +UPDATE 1 +-- three rows +UPDATE multiple_hash SET data = data ||'-2' WHERE category = '1'; +UPDATE 3 +-- three rows, with RETURNING +UPDATE multiple_hash SET data = data ||'-2' WHERE category = '1' RETURNING category; + category +---------- + 1 + 1 + 1 +(3 rows) + +UPDATE 3 +-- check +SELECT * FROM multiple_hash WHERE category = '1' ORDER BY category, data; + category | data +----------+--------- + 1 | 1-1-2-2 + 1 | 2-2-2 + 1 | 3-2-2 +(3 rows) + +-- check that deletes return the right number of rows +-- one row +DELETE FROM multiple_hash WHERE category = '2' AND data = '1'; +DELETE 1 +-- two rows +DELETE FROM multiple_hash WHERE category = '2'; +DELETE 3 +-- three rows, with RETURNING +DELETE FROM multiple_hash WHERE category = '1' RETURNING category; + category +---------- + 1 + 1 + 1 +(3 rows) + +DELETE 3 +-- check +SELECT * FROM multiple_hash WHERE category = '1' ORDER BY category, data; + category | data +----------+------ +(0 rows) + +SELECT * FROM multiple_hash WHERE category = '2' ORDER BY category, data; + category | data +----------+------ +(0 rows) + +-- verify interaction of default values, SERIAL, and RETURNING +\set QUIET on +CREATE TABLE app_analytics_events (id serial, app_id integer, name text); +SET citus.shard_count TO 4; +SELECT create_distributed_table('app_analytics_events', 'app_id', 'hash'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO app_analytics_events VALUES (DEFAULT, 101, 'Fauxkemon Geaux') RETURNING id; + id +---- + 1 +(1 row) + +INSERT INTO app_analytics_events (app_id, name) VALUES (102, 'Wayz') RETURNING id; + id +---- + 2 +(1 row) + +INSERT INTO app_analytics_events (app_id, name) VALUES (103, 'Mynt') RETURNING *; + id | app_id | name +----+--------+------ + 3 | 103 | Mynt +(1 row) + +DROP TABLE app_analytics_events; +-- again with serial in the partition column +CREATE TABLE app_analytics_events (id serial, app_id integer, name text); +SELECT create_distributed_table('app_analytics_events', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO app_analytics_events VALUES (DEFAULT, 101, 'Fauxkemon Geaux') RETURNING id; + id +---- + 1 +(1 row) + +INSERT INTO app_analytics_events (app_id, name) VALUES (102, 'Wayz') RETURNING id; + id +---- + 2 +(1 row) + +INSERT INTO app_analytics_events (app_id, name) VALUES (103, 'Mynt') RETURNING *; + id | app_id | name +----+--------+------ + 3 | 103 | Mynt +(1 row) + +-- Test multi-row insert with serial in the partition column +INSERT INTO app_analytics_events (app_id, name) +VALUES (104, 'Wayz'), (105, 'Mynt') RETURNING *; + id | app_id | name +----+--------+------ + 4 | 104 | Wayz + 5 | 105 | Mynt +(2 rows) + +INSERT INTO app_analytics_events (id, name) +VALUES (DEFAULT, 'Foo'), (300, 'Wah') RETURNING *; + id | app_id | name +-----+--------+------ + 6 | | Foo + 300 | | Wah +(2 rows) + +PREPARE prep(varchar) AS +INSERT INTO app_analytics_events (id, name) +VALUES (DEFAULT, $1 || '.1'), (400 , $1 || '.2') RETURNING *; +EXECUTE prep('version-1'); + id | app_id | name +-----+--------+------------- + 7 | | version-1.1 + 400 | | version-1.2 +(2 rows) + +EXECUTE prep('version-2'); + id | app_id | name +-----+--------+------------- + 8 | | version-2.1 + 400 | | version-2.2 +(2 rows) + +EXECUTE prep('version-3'); + id | app_id | name +-----+--------+------------- + 9 | | version-3.1 + 400 | | version-3.2 +(2 rows) + +EXECUTE prep('version-4'); + id | app_id | name +-----+--------+------------- + 10 | | version-4.1 + 400 | | version-4.2 +(2 rows) + +EXECUTE prep('version-5'); + id | app_id | name +-----+--------+------------- + 11 | | version-5.1 + 400 | | version-5.2 +(2 rows) + +EXECUTE prep('version-6'); + id | app_id | name +-----+--------+------------- + 12 | | version-6.1 + 400 | | version-6.2 +(2 rows) + +SELECT * FROM app_analytics_events ORDER BY id, name; + id | app_id | name +-----+--------+----------------- + 1 | 101 | Fauxkemon Geaux + 2 | 102 | Wayz + 3 | 103 | Mynt + 4 | 104 | Wayz + 5 | 105 | Mynt + 6 | | Foo + 7 | | version-1.1 + 8 | | version-2.1 + 9 | | version-3.1 + 10 | | version-4.1 + 11 | | version-5.1 + 12 | | version-6.1 + 300 | | Wah + 400 | | version-1.2 + 400 | | version-2.2 + 400 | | version-3.2 + 400 | | version-4.2 + 400 | | version-5.2 + 400 | | version-6.2 +(19 rows) + +TRUNCATE app_analytics_events; +-- Test multi-row insert with a dropped column +ALTER TABLE app_analytics_events DROP COLUMN app_id; +INSERT INTO app_analytics_events (name) +VALUES ('Wayz'), ('Mynt') RETURNING *; + id | name +----+------ + 13 | Wayz + 14 | Mynt +(2 rows) + +SELECT * FROM app_analytics_events ORDER BY id; + id | name +----+------ + 13 | Wayz + 14 | Mynt +(2 rows) + +DROP TABLE app_analytics_events; +-- Test multi-row insert with a dropped column before the partition column +CREATE TABLE app_analytics_events (id int default 3, app_id integer, name text); +SELECT create_distributed_table('app_analytics_events', 'name', colocate_with => 'none'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE app_analytics_events DROP COLUMN app_id; +INSERT INTO app_analytics_events (name) +VALUES ('Wayz'), ('Mynt') RETURNING *; + id | name +----+------ + 3 | Mynt + 3 | Wayz +(2 rows) + +SELECT * FROM app_analytics_events WHERE name = 'Wayz'; + id | name +----+------ + 3 | Wayz +(1 row) + +DROP TABLE app_analytics_events; +-- Test multi-row insert with serial in a reference table +CREATE TABLE app_analytics_events (id serial, app_id integer, name text); +SELECT create_reference_table('app_analytics_events'); + create_reference_table +------------------------ + +(1 row) + +INSERT INTO app_analytics_events (app_id, name) +VALUES (104, 'Wayz'), (105, 'Mynt') RETURNING *; + id | app_id | name +----+--------+------ + 1 | 104 | Wayz + 2 | 105 | Mynt +(2 rows) + +SELECT * FROM app_analytics_events ORDER BY id; + id | app_id | name +----+--------+------ + 1 | 104 | Wayz + 2 | 105 | Mynt +(2 rows) + +DROP TABLE app_analytics_events; +-- Test multi-row insert with serial in a non-partition column +CREATE TABLE app_analytics_events (id int, app_id serial, name text); +SELECT create_distributed_table('app_analytics_events', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO app_analytics_events (id, name) +VALUES (99, 'Wayz'), (98, 'Mynt') RETURNING name, app_id; + name | app_id +------+-------- + Mynt | 2 + Wayz | 1 +(2 rows) + +SELECT * FROM app_analytics_events ORDER BY id; + id | app_id | name +----+--------+------ + 98 | 2 | Mynt + 99 | 1 | Wayz +(2 rows) + +DROP TABLE app_analytics_events; +-- test UPDATE with subqueries +CREATE TABLE raw_table (id bigint, value bigint); +CREATE TABLE summary_table ( + id bigint, + min_value numeric, + average_value numeric, + count int, + uniques int); +SELECT create_distributed_table('raw_table', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('summary_table', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO raw_table VALUES (1, 100); +INSERT INTO raw_table VALUES (1, 200); +INSERT INTO raw_table VALUES (1, 200); +INSERT INTO raw_table VALUES (1, 300); +INSERT INTO raw_table VALUES (2, 400); +INSERT INTO raw_table VALUES (2, 500); +INSERT INTO summary_table VALUES (1); +INSERT INTO summary_table VALUES (2); +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+---------------+-------+--------- + 1 | | | | + 2 | | | | +(2 rows) + +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 1 + ) average_query +WHERE id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | | 200.0000000000000000 | | + 2 | | | | +(2 rows) + +-- try different syntax +UPDATE summary_table SET (min_value, average_value) = + (SELECT min(value), avg(value) FROM raw_table WHERE id = 2) +WHERE id = 2; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +UPDATE summary_table SET min_value = 100 + WHERE id IN (SELECT id FROM raw_table WHERE id = 1 and value > 100) AND id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- indeed, we don't need filter on UPDATE explicitly if SELECT already prunes to one shard +UPDATE summary_table SET uniques = 2 + WHERE id IN (SELECT id FROM raw_table WHERE id = 1 and value IN (100, 200)); +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | | 2 + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- use inner results for non-partition column +UPDATE summary_table SET uniques = NULL + WHERE min_value IN (SELECT value FROM raw_table WHERE id = 1) AND id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- these should not update anything +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 1 AND id = 4 + ) average_query +WHERE id = 1 AND id = 4; +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 1 + ) average_query +WHERE id = 1 AND id = 4; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- update with NULL value +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 1 AND id = 4 + ) average_query +WHERE id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- multi-shard updates with recursively planned subqueries +BEGIN; +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table) average_query; +ROLLBACK; +BEGIN; +UPDATE summary_table SET average_value = average_value + 1 WHERE id = + (SELECT id FROM raw_table WHERE value > 100 LIMIT 1); +ROLLBACK; +-- test complex queries +UPDATE summary_table +SET + uniques = metrics.expensive_uniques, + count = metrics.total_count +FROM + (SELECT + id, + count(DISTINCT (CASE WHEN value > 100 then value end)) AS expensive_uniques, + count(value) AS total_count + FROM raw_table + WHERE id = 1 + GROUP BY id) metrics +WHERE + summary_table.id = metrics.id AND + summary_table.id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | | 4 | 2 + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- test joins +UPDATE summary_table SET count = count + 1 FROM raw_table + WHERE raw_table.id = summary_table.id AND summary_table.id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | | 5 | 2 + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- test with prepared statements +PREPARE prepared_update_with_subquery(int, int) AS + UPDATE summary_table SET count = count + $1 FROM raw_table + WHERE raw_table.id = summary_table.id AND summary_table.id = $2; +-- execute 6 times to trigger prepared statement usage +EXECUTE prepared_update_with_subquery(10, 1); +EXECUTE prepared_update_with_subquery(10, 1); +EXECUTE prepared_update_with_subquery(10, 1); +EXECUTE prepared_update_with_subquery(10, 1); +EXECUTE prepared_update_with_subquery(10, 1); +EXECUTE prepared_update_with_subquery(10, 1); +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | | 65 | 2 + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- test with reference tables +CREATE TABLE reference_raw_table (id bigint, value bigint); +CREATE TABLE reference_summary_table ( + id bigint, + min_value numeric, + average_value numeric, + count int, + uniques int); +SELECT create_reference_table('reference_raw_table'); + create_reference_table +------------------------ + +(1 row) + +SELECT create_reference_table('reference_summary_table'); + create_reference_table +------------------------ + +(1 row) + +INSERT INTO reference_raw_table VALUES (1, 100); +INSERT INTO reference_raw_table VALUES (1, 200); +INSERT INTO reference_raw_table VALUES (1, 200); +INSERT INTO reference_raw_table VALUES (1,300), (2, 400), (2,500) RETURNING *; + id | value +----+------- + 1 | 300 + 2 | 400 + 2 | 500 +(3 rows) + +INSERT INTO reference_summary_table VALUES (1); +INSERT INTO reference_summary_table VALUES (2); +SELECT * FROM reference_summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+---------------+-------+--------- + 1 | | | | + 2 | | | | +(2 rows) + +UPDATE reference_summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM reference_raw_table WHERE id = 1 + ) average_query +WHERE id = 1; +UPDATE reference_summary_table SET average_value = average_query.average_value FROM ( + SELECT average_value FROM summary_table WHERE id = 1 FOR UPDATE + ) average_query +WHERE id = 1; +ERROR: cannot perform select on a distributed table and modify a reference table +UPDATE reference_summary_table SET (min_value, average_value) = + (SELECT min(value), avg(value) FROM reference_raw_table WHERE id = 2) +WHERE id = 2; +SELECT * FROM reference_summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | | +(2 rows) + +-- no need partition colum equalities on reference tables +UPDATE reference_summary_table SET (count) = + (SELECT id AS inner_id FROM reference_raw_table WHERE value = 500) +WHERE min_value = 400; +SELECT * FROM reference_summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | | 200.0000000000000000 | | + 2 | 400 | 450.0000000000000000 | 2 | +(2 rows) + +-- can read from a reference table and update a distributed table +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM reference_raw_table WHERE id = 1 + ) average_query +WHERE id = 1; +-- cannot read from a distributed table and update a reference table +UPDATE reference_summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 1 + ) average_query +WHERE id = 1; +ERROR: cannot perform select on a distributed table and modify a reference table +UPDATE reference_summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 1 AND id = 2 + ) average_query +WHERE id = 1; +ERROR: cannot perform select on a distributed table and modify a reference table +-- test connection API via using COPY +-- COPY on SELECT part +BEGIN; +\COPY raw_table FROM STDIN WITH CSV +INSERT INTO summary_table VALUES (3); +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 3 + ) average_query +WHERE id = 3; +COMMIT; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | 65 | 2 + 2 | 400 | 450.0000000000000000 | | + 3 | | 150.0000000000000000 | | +(3 rows) + +-- COPY on UPDATE part +BEGIN; +INSERT INTO raw_table VALUES (4, 100); +INSERT INTO raw_table VALUES (4, 200); +\COPY summary_table FROM STDIN WITH CSV +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 4 + ) average_query +WHERE id = 4; +COMMIT; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | 65 | 2 + 2 | 400 | 450.0000000000000000 | | + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | +(4 rows) + +-- COPY on both part +BEGIN; +\COPY raw_table FROM STDIN WITH CSV +\COPY summary_table FROM STDIN WITH CSV +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM raw_table WHERE id = 5 + ) average_query +WHERE id = 5; +COMMIT; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | 65 | 2 + 2 | 400 | 450.0000000000000000 | | + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | + 5 | | 150.0000000000000000 | | +(5 rows) + +-- COPY on reference tables +BEGIN; +\COPY reference_raw_table FROM STDIN WITH CSV +\COPY summary_table FROM STDIN WITH CSV +UPDATE summary_table SET average_value = average_query.average FROM ( + SELECT avg(value) AS average FROM reference_raw_table WHERE id = 6 + ) average_query +WHERE id = 6; +COMMIT; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 1 | 100 | 200.0000000000000000 | 65 | 2 + 2 | 400 | 450.0000000000000000 | | + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | + 5 | | 150.0000000000000000 | | + 6 | | 150.0000000000000000 | | +(6 rows) + +-- test DELETE queries +SELECT * FROM raw_table ORDER BY id, value; + id | value +----+------- + 1 | 100 + 1 | 200 + 1 | 200 + 1 | 300 + 2 | 400 + 2 | 500 + 3 | 100 + 3 | 200 + 4 | 100 + 4 | 200 + 5 | 100 + 5 | 200 +(12 rows) + +DELETE FROM summary_table + WHERE min_value IN (SELECT value FROM raw_table WHERE id = 1) AND id = 1; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 2 | 400 | 450.0000000000000000 | | + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | + 5 | | 150.0000000000000000 | | + 6 | | 150.0000000000000000 | | +(5 rows) + +-- test with different syntax +DELETE FROM summary_table USING raw_table + WHERE summary_table.id = raw_table.id AND raw_table.id = 2; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | + 5 | | 150.0000000000000000 | | + 6 | | 150.0000000000000000 | | +(4 rows) + +-- cannot read from a distributed table and delete from a reference table +DELETE FROM reference_summary_table USING raw_table + WHERE reference_summary_table.id = raw_table.id AND raw_table.id = 3; +ERROR: cannot perform select on a distributed table and modify a reference table +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | + 5 | | 150.0000000000000000 | | + 6 | | 150.0000000000000000 | | +(4 rows) + +-- test connection API via using COPY with DELETEs +BEGIN; +\COPY summary_table FROM STDIN WITH CSV +DELETE FROM summary_table USING raw_table + WHERE summary_table.id = raw_table.id AND raw_table.id = 1; +DELETE FROM summary_table USING reference_raw_table + WHERE summary_table.id = reference_raw_table.id AND reference_raw_table.id = 2; +COMMIT; +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+----------------------+-------+--------- + 3 | | 150.0000000000000000 | | + 4 | | 150.0000000000000000 | | + 5 | | 150.0000000000000000 | | + 6 | | 150.0000000000000000 | | +(4 rows) + +-- test DELETEs with prepared statements +PREPARE prepared_delete_with_join(int) AS + DELETE FROM summary_table USING raw_table + WHERE summary_table.id = raw_table.id AND raw_table.id = $1; +INSERT INTO raw_table VALUES (6, 100); +-- execute 6 times to trigger prepared statement usage +EXECUTE prepared_delete_with_join(1); +EXECUTE prepared_delete_with_join(2); +EXECUTE prepared_delete_with_join(3); +EXECUTE prepared_delete_with_join(4); +EXECUTE prepared_delete_with_join(5); +EXECUTE prepared_delete_with_join(6); +SELECT * FROM summary_table ORDER BY id; + id | min_value | average_value | count | uniques +----+-----------+---------------+-------+--------- +(0 rows) + +-- we don't support subqueries in VALUES clause +INSERT INTO summary_table (id) VALUES ((SELECT id FROM summary_table)); +ERROR: subqueries are not supported within INSERT queries +HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax. +INSERT INTO summary_table (id) VALUES (5), ((SELECT id FROM summary_table)); +ERROR: subqueries are not supported within INSERT queries +HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax. +-- similar queries with reference tables +INSERT INTO reference_summary_table (id) VALUES ((SELECT id FROM summary_table)); +ERROR: subqueries are not supported within INSERT queries +HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax. +INSERT INTO summary_table (id) VALUES ((SELECT id FROM reference_summary_table)); +ERROR: subqueries are not supported within INSERT queries +HINT: Try rewriting your queries with 'INSERT INTO ... SELECT' syntax. +DROP TABLE raw_table; +DROP TABLE summary_table; +DROP TABLE reference_raw_table; +DROP TABLE reference_summary_table; diff --git a/src/test/regress/expected/multi_modifying_xacts.out b/src/test/regress/expected/multi_modifying_xacts.out index 10d76ca95..3dc2db011 100644 --- a/src/test/regress/expected/multi_modifying_xacts.out +++ b/src/test/regress/expected/multi_modifying_xacts.out @@ -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; diff --git a/src/test/regress/expected/multi_modifying_xacts_9.out b/src/test/regress/expected/multi_modifying_xacts_9.out new file mode 100644 index 000000000..c59189392 --- /dev/null +++ b/src/test/regress/expected/multi_modifying_xacts_9.out @@ -0,0 +1,1582 @@ +SET citus.next_shard_id TO 1200000; +SET citus.next_placement_id TO 1200000; +-- =================================================================== +-- test end-to-end modification functionality +-- =================================================================== +CREATE TABLE researchers ( + id bigint NOT NULL, + lab_id int NOT NULL, + name text NOT NULL +); +CREATE TABLE labs ( + id bigint NOT NULL, + name text NOT NULL +); +SELECT master_create_distributed_table('researchers', 'lab_id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('researchers', 2, 2); + master_create_worker_shards +----------------------------- + +(1 row) + +SELECT master_create_distributed_table('labs', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('labs', 1, 1); + master_create_worker_shards +----------------------------- + +(1 row) + +-- might be confusing to have two people in the same lab with the same name +CREATE UNIQUE INDEX avoid_name_confusion_idx ON researchers (lab_id, name); +-- add some data +INSERT INTO researchers VALUES (1, 1, 'Donald Knuth'); +INSERT INTO researchers VALUES (2, 1, 'Niklaus Wirth'); +INSERT INTO researchers VALUES (3, 2, 'Tony Hoare'); +INSERT INTO researchers VALUES (4, 2, 'Kenneth Iverson'); +-- replace a researcher, reusing their id in a multi-row INSERT +BEGIN; +DELETE FROM researchers WHERE lab_id = 1 AND id = 2; +INSERT INTO researchers VALUES (2, 1, 'John Backus'), (12, 1, 'Frances E. Allen'); +COMMIT; +SELECT name FROM researchers WHERE lab_id = 1 AND id % 10 = 2; + name +------------------ + John Backus + Frances E. Allen +(2 rows) + +-- and the other way around +BEGIN; +INSERT INTO researchers VALUES (14, 2, 'Alan Kay'), (15, 2, 'Barbara Liskov'); +DELETE FROM researchers WHERE id = 14 AND lab_id = 2; +ROLLBACK; +-- should have rolled everything back +SELECT * FROM researchers WHERE id = 15 AND lab_id = 2; + id | lab_id | name +----+--------+------ +(0 rows) + +-- abort a modification +BEGIN; +DELETE FROM researchers WHERE lab_id = 1 AND id = 1; +ABORT; +SELECT name FROM researchers WHERE lab_id = 1 AND id = 1; + name +-------------- + Donald Knuth +(1 row) + +-- trigger a unique constraint violation +BEGIN; +\set VERBOSITY TERSE +UPDATE researchers SET name = 'John Backus' WHERE id = 1 AND lab_id = 1; +ERROR: duplicate key value violates unique constraint "avoid_name_confusion_idx_1200000" +\set VERBOSITY DEFAULT +ABORT; +-- creating savepoints should work... +BEGIN; +INSERT INTO researchers VALUES (5, 3, 'Dennis Ritchie'); +SAVEPOINT hire_thompson; +INSERT INTO researchers VALUES (6, 3, 'Ken Thompson'); +COMMIT; +SELECT name FROM researchers WHERE lab_id = 3 AND id = 6; + name +-------------- + Ken Thompson +(1 row) + +-- even if created by PL/pgSQL... +\set VERBOSITY terse +BEGIN; +DO $$ +BEGIN + INSERT INTO researchers VALUES (10, 10, 'Edsger Dijkstra'); +EXCEPTION + WHEN not_null_violation THEN + RAISE NOTICE 'caught not_null_violation'; +END $$; +COMMIT; +-- rollback should also work +BEGIN; +INSERT INTO researchers VALUES (7, 4, 'Jim Gray'); +SAVEPOINT hire_engelbart; +INSERT INTO researchers VALUES (8, 4, 'Douglas Engelbart'); +ROLLBACK TO hire_engelbart; +COMMIT; +SELECT name FROM researchers WHERE lab_id = 4; + name +---------- + Jim Gray +(1 row) + +BEGIN; +DO $$ +BEGIN + INSERT INTO researchers VALUES (11, 11, 'Whitfield Diffie'); + INSERT INTO researchers VALUES (NULL, 10, 'Edsger Dijkstra'); +EXCEPTION + WHEN not_null_violation THEN + RAISE NOTICE 'caught not_null_violation'; +END $$; +NOTICE: caught not_null_violation +COMMIT; +\set VERBOSITY default +-- should be valid to edit labs after researchers... +BEGIN; +INSERT INTO researchers VALUES (8, 5, 'Douglas Engelbart'); +INSERT INTO labs VALUES (5, 'Los Alamos'); +COMMIT; +SELECT * FROM researchers, labs WHERE labs.id = researchers.lab_id AND researchers.lab_id = 5; + id | lab_id | name | id | name +----+--------+-------------------+----+------------ + 8 | 5 | Douglas Engelbart | 5 | Los Alamos +(1 row) + +-- and the other way around is also allowed +BEGIN; +INSERT INTO labs VALUES (6, 'Bell Labs'); +INSERT INTO researchers VALUES (9, 6, 'Leslie Lamport'); +COMMIT; +-- we should be able to expand the transaction participants +BEGIN; +INSERT INTO labs VALUES (6, 'Bell Labs'); +INSERT INTO researchers VALUES (9, 6, 'Leslie Lamport'); +ERROR: duplicate key value violates unique constraint "avoid_name_confusion_idx_1200001" +DETAIL: Key (lab_id, name)=(6, Leslie Lamport) already exists. +CONTEXT: while executing command on localhost:57638 +ABORT; +-- SELECTs may occur after a modification: First check that selecting +-- from the modified node works. +BEGIN; +INSERT INTO labs VALUES (6, 'Bell Labs'); +SELECT count(*) FROM researchers WHERE lab_id = 6; + count +------- + 1 +(1 row) + +ABORT; +-- then check that SELECT going to new node still is fine +BEGIN; +UPDATE pg_dist_shard_placement AS sp SET shardstate = 3 +FROM pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND sp.nodename = 'localhost' +AND sp.nodeport = :worker_1_port +AND s.logicalrelid = 'researchers'::regclass; +INSERT INTO labs VALUES (6, 'Bell Labs'); +SELECT count(*) FROM researchers WHERE lab_id = 6; + count +------- + 1 +(1 row) + +ABORT; +-- we can mix DDL and INSERT +BEGIN; +INSERT INTO labs VALUES (6, 'Bell Labs'); +ALTER TABLE labs ADD COLUMN motto text; +ABORT; +-- whether it occurs first or second +BEGIN; +ALTER TABLE labs ADD COLUMN motto text; +INSERT INTO labs VALUES (6, 'Bell Labs'); +ABORT; +-- this should work find with sequential DDL as well +BEGIN; +SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; +ALTER TABLE labs ADD COLUMN motto text; +INSERT INTO labs VALUES (6, 'Bell Labs'); +ABORT; +-- but the DDL should correctly roll back +SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid='public.labs'::regclass; + Column | Type | Modifiers +--------+--------+----------- + id | bigint | not null + name | text | not null +(2 rows) + +SELECT * FROM labs WHERE id = 6; + id | name +----+----------- + 6 | Bell Labs +(1 row) + +-- COPY can happen after single row INSERT +BEGIN; +INSERT INTO labs VALUES (6, 'Bell Labs'); +\copy labs from stdin delimiter ',' +COMMIT; +-- COPY can be performed if multiple shards were modified over the same connection +BEGIN; +INSERT INTO researchers VALUES (2, 1, 'Knuth Donald'); +INSERT INTO researchers VALUES (10, 6, 'Lamport Leslie'); +\copy researchers from stdin delimiter ',' +ROLLBACK; +-- COPY can be performed after a multi-row INSERT that uses one connection +BEGIN; +INSERT INTO researchers VALUES (2, 1, 'Knuth Donald'), (10, 6, 'Lamport Leslie'); +\copy researchers from stdin delimiter ',' +ROLLBACK; +-- after a COPY you can modify multiple shards, since they'll use different connections +BEGIN; +\copy researchers from stdin delimiter ',' +INSERT INTO researchers VALUES (2, 1, 'Knuth Donald'); +INSERT INTO researchers VALUES (10, 6, 'Lamport Leslie'); +ROLLBACK; +-- after a COPY you can perform a multi-row INSERT +BEGIN; +\copy researchers from stdin delimiter ',' +INSERT INTO researchers VALUES (2, 1, 'Knuth Donald'), (10, 6, 'Lamport Leslie'); +ROLLBACK; +-- COPY can happen before single row INSERT +BEGIN; +\copy labs from stdin delimiter ',' +SELECT name FROM labs WHERE id = 10; + name +---------------- + Weyland-Yutani + Weyland-Yutani +(2 rows) + +INSERT INTO labs VALUES (6, 'Bell Labs'); +COMMIT; +-- two consecutive COPYs in a transaction are allowed +BEGIN; +\copy labs from stdin delimiter ',' +\copy labs from stdin delimiter ',' +COMMIT; +SELECT name FROM labs WHERE id = 11 OR id = 12 ORDER BY id; + name +---------------- + Planet Express + fsociety +(2 rows) + +-- 1pc failure test +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +-- copy with unique index violation +BEGIN; +\copy researchers FROM STDIN delimiter ',' +\copy researchers FROM STDIN delimiter ',' +ERROR: duplicate key value violates unique constraint "avoid_name_confusion_idx_1200001" +DETAIL: Key (lab_id, name)=(6, 'Bjarne Stroustrup') already exists. +COMMIT; +-- verify rollback +SELECT * FROM researchers WHERE lab_id = 6; + id | lab_id | name +----+--------+---------------- + 9 | 6 | Leslie Lamport +(1 row) + +SELECT count(*) FROM pg_dist_transaction; + count +------- + 0 +(1 row) + +-- 2pc failure and success tests +SET citus.multi_shard_commit_protocol TO '2pc'; +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +-- copy with unique index violation +BEGIN; +\copy researchers FROM STDIN delimiter ',' +\copy researchers FROM STDIN delimiter ',' +ERROR: duplicate key value violates unique constraint "avoid_name_confusion_idx_1200001" +DETAIL: Key (lab_id, name)=(6, 'Bjarne Stroustrup') already exists. +COMMIT; +-- verify rollback +SELECT * FROM researchers WHERE lab_id = 6; + id | lab_id | name +----+--------+---------------- + 9 | 6 | Leslie Lamport +(1 row) + +SELECT count(*) FROM pg_dist_transaction; + count +------- + 0 +(1 row) + +BEGIN; +\copy researchers FROM STDIN delimiter ',' +\copy researchers FROM STDIN delimiter ',' +COMMIT; +-- verify success +SELECT * FROM researchers WHERE lab_id = 6; + id | lab_id | name +----+--------+---------------------- + 9 | 6 | Leslie Lamport + 17 | 6 | 'Bjarne Stroustrup' + 18 | 6 | 'Dennis Ritchie' +(3 rows) + +-- verify 2pc +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +RESET citus.multi_shard_commit_protocol; +-- create a check function +SELECT * from run_command_on_workers('CREATE FUNCTION reject_large_id() RETURNS trigger AS $rli$ + BEGIN + IF (NEW.id > 30) THEN + RAISE ''illegal value''; + END IF; + + RETURN NEW; + END; +$rli$ LANGUAGE plpgsql;') +ORDER BY nodeport; + nodename | nodeport | success | result +-----------+----------+---------+----------------- + localhost | 57637 | t | CREATE FUNCTION + localhost | 57638 | t | CREATE FUNCTION +(2 rows) + +-- register after insert trigger +SELECT * FROM run_command_on_placements('researchers', 'CREATE CONSTRAINT TRIGGER reject_large_researcher_id AFTER INSERT ON %s DEFERRABLE INITIALLY DEFERRED FOR EACH ROW EXECUTE PROCEDURE reject_large_id()') +ORDER BY nodeport, shardid; + nodename | nodeport | shardid | success | result +-----------+----------+---------+---------+---------------- + localhost | 57637 | 1200000 | t | CREATE TRIGGER + localhost | 57637 | 1200001 | t | CREATE TRIGGER + localhost | 57638 | 1200000 | t | CREATE TRIGGER + localhost | 57638 | 1200001 | t | CREATE TRIGGER +(4 rows) + +-- hide postgresql version dependend messages for next test only +\set VERBOSITY terse +-- deferred check should abort the transaction +BEGIN; +SET LOCAL citus.multi_shard_commit_protocol TO '1pc'; +DELETE FROM researchers WHERE lab_id = 6; +\copy researchers FROM STDIN delimiter ',' +\copy researchers FROM STDIN delimiter ',' +COMMIT; +WARNING: illegal value +WARNING: failed to commit transaction on localhost:57638 +WARNING: illegal value +WARNING: failed to commit transaction on localhost:57637 +WARNING: could not commit transaction for shard 1200001 on any active node +ERROR: could not commit transaction on any active node +\unset VERBOSITY +-- verify everyhing including delete is rolled back +SELECT * FROM researchers WHERE lab_id = 6; + id | lab_id | name +----+--------+---------------------- + 9 | 6 | Leslie Lamport + 17 | 6 | 'Bjarne Stroustrup' + 18 | 6 | 'Dennis Ritchie' +(3 rows) + +-- cleanup triggers and the function +SELECT * from run_command_on_placements('researchers', 'drop trigger reject_large_researcher_id on %s') +ORDER BY nodeport, shardid; + nodename | nodeport | shardid | success | result +-----------+----------+---------+---------+-------------- + localhost | 57637 | 1200000 | t | DROP TRIGGER + localhost | 57637 | 1200001 | t | DROP TRIGGER + localhost | 57638 | 1200000 | t | DROP TRIGGER + localhost | 57638 | 1200001 | t | DROP TRIGGER +(4 rows) + +SELECT * FROM run_command_on_workers('drop function reject_large_id()') +ORDER BY nodeport; + nodename | nodeport | success | result +-----------+----------+---------+--------------- + localhost | 57637 | t | DROP FUNCTION + localhost | 57638 | t | DROP FUNCTION +(2 rows) + +-- ALTER and copy are compatible +BEGIN; +ALTER TABLE labs ADD COLUMN motto text; +\copy labs from stdin delimiter ',' +ROLLBACK; +BEGIN; +\copy labs from stdin delimiter ',' +ALTER TABLE labs ADD COLUMN motto text; +ABORT; +-- 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 +-------- +(0 rows) + +SELECT lab_id FROM researchers WHERE lab_id = 2 AND id = 0; + lab_id +-------- +(0 rows) + +ALTER TABLE researchers ADD COLUMN motto text; +ROLLBACK; +-- can perform sequential DDL once a connection is used for multiple shards +BEGIN; +SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; +SELECT lab_id FROM researchers WHERE lab_id = 1 AND id = 0; + lab_id +-------- +(0 rows) + +SELECT lab_id FROM researchers WHERE lab_id = 2 AND id = 0; + lab_id +-------- +(0 rows) + +ALTER TABLE researchers ADD COLUMN motto text; +ROLLBACK; +-- multi-shard operations can co-exist with DDL in a transactional way +BEGIN; +ALTER TABLE labs ADD COLUMN motto text; +DELETE FROM labs; +ALTER TABLE labs ADD COLUMN score float; +ROLLBACK; +-- should have rolled everything back +SELECT * FROM labs WHERE id = 12; + id | name +----+---------- + 12 | fsociety +(1 row) + +-- now, for some special failures... +CREATE TABLE objects ( + id bigint PRIMARY KEY, + name text NOT NULL +); +SELECT master_create_distributed_table('objects', 'id', 'hash'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SELECT master_create_worker_shards('objects', 1, 2); + master_create_worker_shards +----------------------------- + +(1 row) + +-- test primary key violations +BEGIN; +INSERT INTO objects VALUES (1, 'apple'); +INSERT INTO objects VALUES (1, 'orange'); +ERROR: duplicate key value violates unique constraint "objects_pkey_1200003" +DETAIL: Key (id)=(1) already exists. +CONTEXT: while executing command on localhost:57637 +COMMIT; +-- data shouldn't have persisted... +SELECT * FROM objects WHERE id = 1; + id | name +----+------ +(0 rows) + +-- and placements should still be healthy... +SELECT count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND sp.shardstate = 1 +AND s.logicalrelid = 'objects'::regclass; + count +------- + 2 +(1 row) + +-- create trigger on one worker to reject certain values +\c - - - :worker_2_port +CREATE FUNCTION reject_bad() RETURNS trigger AS $rb$ + BEGIN + IF (NEW.name = 'BAD') THEN + RAISE 'illegal value'; + END IF; + + RETURN NEW; + END; +$rb$ LANGUAGE plpgsql; +CREATE CONSTRAINT TRIGGER reject_bad +AFTER INSERT ON objects_1200003 +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'); +ERROR: illegal value +COMMIT; +-- so the data should noy be persisted +SELECT * FROM objects WHERE id = 2; + id | name +----+------ +(0 rows) + +SELECT * FROM labs WHERE id = 7; + id | name +----+------ +(0 rows) + +-- and none of placements should be inactive +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 = 'objects'::regclass; + count +------- + 0 +(1 row) + +DELETE FROM objects; +-- 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 + IF (NEW.name = 'BAD') THEN + RAISE 'illegal value'; + END IF; + + RETURN NEW; + END; +$rb$ LANGUAGE plpgsql; +CREATE CONSTRAINT TRIGGER reject_bad +AFTER INSERT ON labs_1200002 +DEFERRABLE INITIALLY IMMEDIATE +FOR EACH ROW EXECUTE PROCEDURE reject_bad(); +\c - - - :master_port +BEGIN; +INSERT INTO objects VALUES (1, 'apple'); +INSERT INTO objects VALUES (2, 'BAD'); +ERROR: illegal value +INSERT INTO labs VALUES (8, 'Aperture Science'); +ERROR: current transaction is aborted, commands ignored until end of transaction block +INSERT INTO labs VALUES (2, 'BAD'); +ERROR: current transaction is aborted, commands ignored until end of transaction block +COMMIT; +-- data should NOT be persisted +SELECT * FROM objects WHERE id = 1; + id | name +----+------ +(0 rows) + +SELECT * FROM labs WHERE id = 8; + id | name +----+------ +(0 rows) + +-- all placements should remain healthy +SELECT count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND sp.shardstate = 1 +AND (s.logicalrelid = 'objects'::regclass OR + s.logicalrelid = 'labs'::regclass); + count +------- + 3 +(1 row) + +-- what if the failures happen at COMMIT time? +\c - - - :worker_2_port +DROP TRIGGER reject_bad ON objects_1200003; +CREATE CONSTRAINT TRIGGER reject_bad +AFTER INSERT ON objects_1200003 +DEFERRABLE INITIALLY DEFERRED +FOR EACH ROW EXECUTE PROCEDURE reject_bad(); +\c - - - :master_port +-- should be the same story as before, just at COMMIT time +BEGIN; +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 +----+------ + 2 | BAD +(1 row) + +-- but one placement should be bad +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 = 'objects'::regclass; + count +------- + 1 +(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 all nodes have failures at COMMIT time? +\c - - - :worker_1_port +DROP TRIGGER reject_bad ON labs_1200002; +CREATE CONSTRAINT TRIGGER reject_bad +AFTER INSERT ON labs_1200002 +DEFERRABLE INITIALLY DEFERRED +FOR EACH ROW EXECUTE PROCEDURE reject_bad(); +\c - - - :master_port +BEGIN; +INSERT INTO objects VALUES (1, 'apple'); +INSERT INTO objects VALUES (2, 'BAD'); +INSERT INTO labs VALUES (8, 'Aperture Science'); +INSERT INTO labs VALUES (9, 'BAD'); +COMMIT; +WARNING: illegal value +WARNING: failed to commit transaction on localhost:57637 +WARNING: illegal value +WARNING: failed to commit transaction on localhost:57638 +WARNING: could not commit transaction for shard 1200002 on any active node +WARNING: could not commit transaction for shard 1200003 on any active node +ERROR: could not commit transaction on any active node +-- data should NOT be persisted +SELECT * FROM objects WHERE id = 1; + id | name +----+------ +(0 rows) + +SELECT * FROM labs WHERE id = 8; + id | name +----+------ +(0 rows) + +-- all placements should remain healthy +SELECT count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND sp.shardstate = 1 +AND (s.logicalrelid = 'objects'::regclass OR + s.logicalrelid = 'labs'::regclass); + count +------- + 3 +(1 row) + +-- what if one shard (objects) succeeds but another (labs) completely fails? +\c - - - :worker_2_port +DROP TRIGGER reject_bad ON objects_1200003; +\c - - - :master_port +SET citus.next_shard_id TO 1200004; +BEGIN; +INSERT INTO objects VALUES (1, 'apple'); +INSERT INTO labs VALUES (8, 'Aperture Science'); +INSERT INTO labs VALUES (9, 'BAD'); +COMMIT; +WARNING: illegal value +WARNING: failed to commit transaction on localhost:57637 +WARNING: could not commit transaction for shard 1200002 on any active node +\set VERBOSITY default +-- data to objects should be persisted, but labs should not... +SELECT * FROM objects WHERE id = 1; + id | name +----+------- + 1 | apple +(1 row) + +SELECT * FROM labs WHERE id = 8; + id | name +----+------ +(0 rows) + +-- labs should be healthy, but one object placement shouldn't be +SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND (s.logicalrelid = 'objects'::regclass OR + s.logicalrelid = 'labs'::regclass) +GROUP BY s.logicalrelid, sp.shardstate +ORDER BY s.logicalrelid, sp.shardstate; + logicalrelid | shardstate | count +--------------+------------+------- + labs | 1 | 1 + objects | 1 | 1 + objects | 3 | 1 +(3 rows) + +-- some append-partitioned tests for good measure +CREATE TABLE append_researchers ( LIKE researchers ); +SELECT master_create_distributed_table('append_researchers', 'id', 'append'); + master_create_distributed_table +--------------------------------- + +(1 row) + +SET citus.shard_replication_factor TO 1; +SELECT master_create_empty_shard('append_researchers') AS new_shard_id +\gset +UPDATE pg_dist_shard SET shardminvalue = 0, shardmaxvalue = 500000 +WHERE shardid = :new_shard_id; +SELECT master_create_empty_shard('append_researchers') AS new_shard_id +\gset +UPDATE pg_dist_shard SET shardminvalue = 500000, shardmaxvalue = 1000000 +WHERE shardid = :new_shard_id; +SET citus.shard_replication_factor TO DEFAULT; +-- try single-shard INSERT +BEGIN; +INSERT INTO append_researchers VALUES (0, 0, 'John Backus'); +COMMIT; +SELECT * FROM append_researchers WHERE id = 0; + id | lab_id | name +----+--------+------------- + 0 | 0 | John Backus +(1 row) + +-- try rollback +BEGIN; +DELETE FROM append_researchers WHERE id = 0; +ROLLBACK; +SELECT * FROM append_researchers WHERE id = 0; + id | lab_id | name +----+--------+------------- + 0 | 0 | John Backus +(1 row) + +-- try hitting shard on other node +BEGIN; +INSERT INTO append_researchers VALUES (1, 1, 'John McCarthy'); +INSERT INTO append_researchers VALUES (500000, 500000, 'Tony Hoare'); +ERROR: cannot run INSERT command which targets multiple shards +HINT: Make sure the value for partition column "id" falls into a single shard. +ROLLBACK; +SELECT * FROM append_researchers; + id | lab_id | name +----+--------+------------- + 0 | 0 | John Backus +(1 row) + +-- we use 2PC for reference tables by default +-- let's add some tests for them +CREATE TABLE reference_modifying_xacts (key int, value int); +SELECT create_reference_table('reference_modifying_xacts'); + create_reference_table +------------------------ + +(1 row) + +-- very basic test, ensure that INSERTs work +INSERT INTO reference_modifying_xacts VALUES (1, 1); +SELECT * FROM reference_modifying_xacts; + key | value +-----+------- + 1 | 1 +(1 row) + +-- now ensure that it works in a transaction as well +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (2, 2); +SELECT * FROM reference_modifying_xacts; + key | value +-----+------- + 1 | 1 + 2 | 2 +(2 rows) + +COMMIT; +-- we should be able to see the insert outside of the transaction as well +SELECT * FROM reference_modifying_xacts; + key | value +-----+------- + 1 | 1 + 2 | 2 +(2 rows) + +-- rollback should also work +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (3, 3); +SELECT * FROM reference_modifying_xacts; + key | value +-----+------- + 1 | 1 + 2 | 2 + 3 | 3 +(3 rows) + +ROLLBACK; +-- see that we've not inserted +SELECT * FROM reference_modifying_xacts; + key | value +-----+------- + 1 | 1 + 2 | 2 +(2 rows) + +-- lets fail on of the workers at before the commit time +\c - - - :worker_1_port +CREATE FUNCTION reject_bad_reference() RETURNS trigger AS $rb$ + BEGIN + IF (NEW.key = 999) THEN + RAISE 'illegal value'; + END IF; + + RETURN NEW; + END; +$rb$ LANGUAGE plpgsql; +CREATE CONSTRAINT TRIGGER reject_bad_reference +AFTER INSERT ON reference_modifying_xacts_1200006 +DEFERRABLE INITIALLY IMMEDIATE +FOR EACH ROW EXECUTE PROCEDURE reject_bad_reference(); +\c - - - :master_port +\set VERBOSITY terse +-- try without wrapping inside a transaction +INSERT INTO reference_modifying_xacts VALUES (999, 3); +ERROR: illegal value +-- same test within a transaction +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (999, 3); +ERROR: illegal value +COMMIT; +-- lets fail one of the workers at COMMIT time +\c - - - :worker_1_port +DROP TRIGGER reject_bad_reference ON reference_modifying_xacts_1200006; +CREATE CONSTRAINT TRIGGER reject_bad_reference +AFTER INSERT ON reference_modifying_xacts_1200006 +DEFERRABLE INITIALLY DEFERRED +FOR EACH ROW EXECUTE PROCEDURE reject_bad_reference(); +\c - - - :master_port +\set VERBOSITY terse +-- try without wrapping inside a transaction +INSERT INTO reference_modifying_xacts VALUES (999, 3); +ERROR: illegal value +-- same test within a transaction +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (999, 3); +COMMIT; +ERROR: illegal value +-- all placements should be healthy +SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND s.logicalrelid = 'reference_modifying_xacts'::regclass +GROUP BY s.logicalrelid, sp.shardstate +ORDER BY s.logicalrelid, sp.shardstate; + logicalrelid | shardstate | count +---------------------------+------------+------- + reference_modifying_xacts | 1 | 2 +(1 row) + +-- for the time-being drop the constraint +\c - - - :worker_1_port +DROP TRIGGER reject_bad_reference ON reference_modifying_xacts_1200006; +\c - - - :master_port +-- now create a hash distributed table and run tests +-- including both the reference table and the hash +-- distributed table +-- To prevent colocating a hash table with append table +DELETE FROM pg_dist_colocation WHERE colocationid = 100001; +SET citus.next_shard_id TO 1200007; +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 1; +CREATE TABLE hash_modifying_xacts (key int, value int); +SELECT create_distributed_table('hash_modifying_xacts', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +-- let's try to expand the xact participants +BEGIN; +INSERT INTO hash_modifying_xacts VALUES (1, 1); +INSERT INTO reference_modifying_xacts VALUES (10, 10); +COMMIT; +-- it is allowed when turning off deadlock prevention +BEGIN; +INSERT INTO hash_modifying_xacts VALUES (1, 1); +INSERT INTO reference_modifying_xacts VALUES (10, 10); +ABORT; +BEGIN; +INSERT INTO hash_modifying_xacts VALUES (1, 1); +INSERT INTO hash_modifying_xacts VALUES (2, 2); +ABORT; +-- lets fail one of the workers before COMMIT time for the hash table +\c - - - :worker_1_port +CREATE FUNCTION reject_bad_hash() RETURNS trigger AS $rb$ + BEGIN + IF (NEW.key = 997) THEN + RAISE 'illegal value'; + END IF; + + RETURN NEW; + END; +$rb$ LANGUAGE plpgsql; +CREATE CONSTRAINT TRIGGER reject_bad_hash +AFTER INSERT ON hash_modifying_xacts_1200007 +DEFERRABLE INITIALLY IMMEDIATE +FOR EACH ROW EXECUTE PROCEDURE reject_bad_hash(); +\c - - - :master_port +\set VERBOSITY terse +-- the transaction as a whole should fail +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (55, 10); +INSERT INTO hash_modifying_xacts VALUES (997, 1); +ERROR: illegal value +COMMIT; +-- ensure that the value didn't go into the reference table +SELECT * FROM reference_modifying_xacts WHERE key = 55; + key | value +-----+------- +(0 rows) + +-- now lets fail on of the workers for the hash distributed table table +-- when there is a reference table involved +\c - - - :worker_1_port +DROP TRIGGER reject_bad_hash ON hash_modifying_xacts_1200007; +-- the trigger is on execution time +CREATE CONSTRAINT TRIGGER reject_bad_hash +AFTER INSERT ON hash_modifying_xacts_1200007 +DEFERRABLE INITIALLY DEFERRED +FOR EACH ROW EXECUTE PROCEDURE reject_bad_hash(); +\c - - - :master_port +\set VERBOSITY terse +-- the transaction as a whole should fail +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (12, 12); +INSERT INTO hash_modifying_xacts VALUES (997, 1); +COMMIT; +ERROR: illegal value +-- ensure that the values didn't go into the reference table +SELECT * FROM reference_modifying_xacts WHERE key = 12; + key | value +-----+------- +(0 rows) + +-- all placements should be healthy +SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND (s.logicalrelid = 'reference_modifying_xacts'::regclass OR + s.logicalrelid = 'hash_modifying_xacts'::regclass) +GROUP BY s.logicalrelid, sp.shardstate +ORDER BY s.logicalrelid, sp.shardstate; + logicalrelid | shardstate | count +---------------------------+------------+------- + reference_modifying_xacts | 1 | 2 + hash_modifying_xacts | 1 | 4 +(2 rows) + +-- now, fail the insert on reference table +-- and ensure that hash distributed table's +-- change is rollbacked as well +\c - - - :worker_1_port +CREATE CONSTRAINT TRIGGER reject_bad_reference +AFTER INSERT ON reference_modifying_xacts_1200006 +DEFERRABLE INITIALLY IMMEDIATE +FOR EACH ROW EXECUTE PROCEDURE reject_bad_reference(); +\c - - - :master_port +\set VERBOSITY terse +BEGIN; +-- to expand participant to include all worker nodes +INSERT INTO reference_modifying_xacts VALUES (66, 3); +INSERT INTO hash_modifying_xacts VALUES (80, 1); +INSERT INTO reference_modifying_xacts VALUES (999, 3); +ERROR: illegal value +COMMIT; +SELECT * FROM hash_modifying_xacts WHERE key = 80; + key | value +-----+------- +(0 rows) + +SELECT * FROM reference_modifying_xacts WHERE key = 66; + key | value +-----+------- +(0 rows) + +SELECT * FROM reference_modifying_xacts WHERE key = 999; + key | value +-----+------- +(0 rows) + +-- all placements should be healthy +SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND (s.logicalrelid = 'reference_modifying_xacts'::regclass OR + s.logicalrelid = 'hash_modifying_xacts'::regclass) +GROUP BY s.logicalrelid, sp.shardstate +ORDER BY s.logicalrelid, sp.shardstate; + logicalrelid | shardstate | count +---------------------------+------------+------- + reference_modifying_xacts | 1 | 2 + hash_modifying_xacts | 1 | 4 +(2 rows) + +-- now show that all modifications to reference +-- tables are done in 2PC +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +INSERT INTO reference_modifying_xacts VALUES (70, 70); +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +-- reset the transactions table +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +BEGIN; +INSERT INTO reference_modifying_xacts VALUES (71, 71); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +-- create a hash distributed tablw which spans all nodes +SET citus.shard_count = 4; +SET citus.shard_replication_factor = 2; +CREATE TABLE hash_modifying_xacts_second (key int, value int); +SELECT create_distributed_table('hash_modifying_xacts_second', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +-- reset the transactions table +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +BEGIN; +INSERT INTO hash_modifying_xacts_second VALUES (72, 1); +INSERT INTO reference_modifying_xacts VALUES (72, 3); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +-- reset the transactions table +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +DELETE FROM reference_modifying_xacts; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +-- reset the transactions table +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +UPDATE reference_modifying_xacts SET key = 10; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +-- now to one more type of failure testing +-- in which we'll make the remote host unavailable +-- first create the new user on all nodes +CREATE USER test_user; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +\c - - - :worker_1_port +CREATE USER test_user; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +\c - - - :worker_2_port +CREATE USER test_user; +NOTICE: not propagating CREATE ROLE/USER commands to worker nodes +-- now connect back to the master with the new user +\c - test_user - :master_port +SET citus.next_shard_id TO 1200015; +CREATE TABLE reference_failure_test (key int, value int); +SELECT create_reference_table('reference_failure_test'); + create_reference_table +------------------------ + +(1 row) + +-- create a hash distributed table +SET citus.shard_count TO 4; +CREATE TABLE numbers_hash_failure_test(key int, value int); +SELECT create_distributed_table('numbers_hash_failure_test', 'key'); + create_distributed_table +-------------------------- + +(1 row) + +-- ensure that the shard is created for this user +\c - test_user - :worker_1_port +\dt reference_failure_test_1200015 + List of relations + Schema | Name | Type | Owner +--------+--------------------------------+-------+----------- + public | reference_failure_test_1200015 | table | test_user +(1 row) + +-- now connect with the default user, +-- and rename the existing user +\c - :default_user - :worker_1_port +ALTER USER test_user RENAME TO test_user_new; +-- connect back to master and query the reference table + \c - test_user - :master_port +-- should fail since the worker doesn't have test_user anymore +INSERT INTO reference_failure_test VALUES (1, '1'); +ERROR: connection error: localhost:57637 +-- the same as the above, but wrapped within a transaction +BEGIN; +INSERT INTO reference_failure_test VALUES (1, '1'); +ERROR: connection error: localhost:57637 +COMMIT; +BEGIN; +COPY reference_failure_test FROM STDIN WITH (FORMAT 'csv'); +ERROR: connection error: localhost:57637 +COMMIT; +-- show that no data go through the table and shard states are good +SELECT * FROM reference_failure_test; + key | value +-----+------- +(0 rows) + +-- all placements should be healthy +SELECT s.logicalrelid::regclass::text, sp.shardstate, count(*) +FROM pg_dist_shard_placement AS sp, + pg_dist_shard AS s +WHERE sp.shardid = s.shardid +AND s.logicalrelid = 'reference_failure_test'::regclass +GROUP BY s.logicalrelid, sp.shardstate +ORDER BY s.logicalrelid, sp.shardstate; + logicalrelid | shardstate | count +------------------------+------------+------- + reference_failure_test | 1 | 2 +(1 row) + +BEGIN; +COPY numbers_hash_failure_test FROM STDIN WITH (FORMAT 'csv'); +WARNING: connection error: localhost:57637 +WARNING: connection error: localhost:57637 +-- some placements are invalid before abort +SELECT shardid, shardstate, nodename, nodeport +FROM pg_dist_shard_placement JOIN pg_dist_shard USING (shardid) +WHERE logicalrelid = 'numbers_hash_failure_test'::regclass +ORDER BY shardid, nodeport; + shardid | shardstate | nodename | nodeport +---------+------------+-----------+---------- + 1200016 | 3 | localhost | 57637 + 1200016 | 1 | localhost | 57638 + 1200017 | 1 | localhost | 57637 + 1200017 | 1 | localhost | 57638 + 1200018 | 1 | localhost | 57637 + 1200018 | 1 | localhost | 57638 + 1200019 | 3 | localhost | 57637 + 1200019 | 1 | localhost | 57638 +(8 rows) + +ABORT; +-- verify nothing is inserted +SELECT count(*) FROM numbers_hash_failure_test; +WARNING: connection error: localhost:57637 +WARNING: connection error: localhost:57637 + count +------- + 0 +(1 row) + +-- all placements to be market valid +SELECT shardid, shardstate, nodename, nodeport +FROM pg_dist_shard_placement JOIN pg_dist_shard USING (shardid) +WHERE logicalrelid = 'numbers_hash_failure_test'::regclass +ORDER BY shardid, nodeport; + shardid | shardstate | nodename | nodeport +---------+------------+-----------+---------- + 1200016 | 1 | localhost | 57637 + 1200016 | 1 | localhost | 57638 + 1200017 | 1 | localhost | 57637 + 1200017 | 1 | localhost | 57638 + 1200018 | 1 | localhost | 57637 + 1200018 | 1 | localhost | 57638 + 1200019 | 1 | localhost | 57637 + 1200019 | 1 | localhost | 57638 +(8 rows) + +BEGIN; +COPY numbers_hash_failure_test FROM STDIN WITH (FORMAT 'csv'); +WARNING: connection error: localhost:57637 +WARNING: connection error: localhost:57637 +-- check shard states before commit +SELECT shardid, shardstate, nodename, nodeport +FROM pg_dist_shard_placement JOIN pg_dist_shard USING (shardid) +WHERE logicalrelid = 'numbers_hash_failure_test'::regclass +ORDER BY shardid, nodeport; + shardid | shardstate | nodename | nodeport +---------+------------+-----------+---------- + 1200016 | 3 | localhost | 57637 + 1200016 | 1 | localhost | 57638 + 1200017 | 1 | localhost | 57637 + 1200017 | 1 | localhost | 57638 + 1200018 | 1 | localhost | 57637 + 1200018 | 1 | localhost | 57638 + 1200019 | 3 | localhost | 57637 + 1200019 | 1 | localhost | 57638 +(8 rows) + +COMMIT; +-- expect some placements to be market invalid after commit +SELECT shardid, shardstate, nodename, nodeport +FROM pg_dist_shard_placement JOIN pg_dist_shard USING (shardid) +WHERE logicalrelid = 'numbers_hash_failure_test'::regclass +ORDER BY shardid, nodeport; + shardid | shardstate | nodename | nodeport +---------+------------+-----------+---------- + 1200016 | 3 | localhost | 57637 + 1200016 | 1 | localhost | 57638 + 1200017 | 1 | localhost | 57637 + 1200017 | 1 | localhost | 57638 + 1200018 | 1 | localhost | 57637 + 1200018 | 1 | localhost | 57638 + 1200019 | 3 | localhost | 57637 + 1200019 | 1 | localhost | 57638 +(8 rows) + +-- verify data is inserted +SELECT count(*) FROM numbers_hash_failure_test; +WARNING: connection error: localhost:57637 +WARNING: connection error: localhost:57637 + count +------- + 2 +(1 row) + +-- break the other node as well +\c - :default_user - :worker_2_port +ALTER USER test_user RENAME TO test_user_new; +\c - test_user - :master_port +-- fails on all shard placements +INSERT INTO numbers_hash_failure_test VALUES (2,2); +ERROR: connection error: localhost:57638 +-- connect back to the master with the proper user to continue the tests +\c - :default_user - :master_port +SET citus.next_shard_id TO 1200020; +SET citus.next_placement_id TO 1200033; +-- unbreak both nodes by renaming the user back to the original name +SELECT * FROM run_command_on_workers('ALTER USER test_user_new RENAME TO test_user'); + nodename | nodeport | success | result +-----------+----------+---------+------------ + localhost | 57637 | t | ALTER ROLE + localhost | 57638 | t | ALTER ROLE +(2 rows) + +DROP TABLE reference_modifying_xacts, hash_modifying_xacts, hash_modifying_xacts_second, + reference_failure_test, numbers_hash_failure_test; +SELECT * FROM run_command_on_workers('DROP USER test_user'); + nodename | nodeport | success | result +-----------+----------+---------+----------- + localhost | 57637 | t | DROP ROLE + localhost | 57638 | t | DROP ROLE +(2 rows) + +DROP USER test_user; +-- set up foreign keys to test transactions with co-located and reference tables +BEGIN; +SET LOCAL citus.shard_replication_factor TO 1; +SET LOCAL citus.shard_count TO 4; +CREATE TABLE usergroups ( + gid int PRIMARY KEY, + name text +); +SELECT create_reference_table('usergroups'); + create_reference_table +------------------------ + +(1 row) + +CREATE TABLE itemgroups ( + gid int PRIMARY KEY, + name text +); +SELECT create_reference_table('itemgroups'); + create_reference_table +------------------------ + +(1 row) + +CREATE TABLE users ( + id int PRIMARY KEY, + name text, + user_group int +); +SELECT create_distributed_table('users', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE items ( + user_id int REFERENCES users (id) ON DELETE CASCADE, + item_name text, + item_group int +); +SELECT create_distributed_table('items', 'user_id'); + create_distributed_table +-------------------------- + +(1 row) + +-- Table to find values that live in different shards on the same node +SELECT id, shard_name('users', shardid), nodename, nodeport +FROM + pg_dist_shard_placement +JOIN + ( SELECT id, get_shard_id_for_distribution_column('users', id) shardid FROM generate_series(1,10) id ) ids +USING (shardid) +ORDER BY + id; + id | shard_name | nodename | nodeport +----+---------------+-----------+---------- + 1 | users_1200022 | localhost | 57637 + 2 | users_1200025 | localhost | 57638 + 3 | users_1200023 | localhost | 57638 + 4 | users_1200023 | localhost | 57638 + 5 | users_1200022 | localhost | 57637 + 6 | users_1200024 | localhost | 57637 + 7 | users_1200023 | localhost | 57638 + 8 | users_1200022 | localhost | 57637 + 9 | users_1200025 | localhost | 57638 + 10 | users_1200022 | localhost | 57637 +(10 rows) + +END; +-- the INSERTs into items should see the users +BEGIN; +\COPY users FROM STDIN WITH CSV +INSERT INTO items VALUES (1, 'item-1'); +INSERT INTO items VALUES (6, 'item-6'); +END; +SELECT user_id FROM items ORDER BY user_id; + user_id +--------- + 1 + 6 +(2 rows) + +-- should be able to open multiple connections per node after INSERTing over one connection +BEGIN; +INSERT INTO users VALUES (2, 'burak'); +INSERT INTO users VALUES (3, 'burak'); +\COPY items FROM STDIN WITH CSV +ROLLBACK; +-- perform parallel DDL after a co-located table has been read over 1 connection +BEGIN; +SELECT id FROM users WHERE id = 1; + id +---- + 1 +(1 row) + +SELECT id FROM users WHERE id = 6; + id +---- + 6 +(1 row) + +ALTER TABLE items ADD COLUMN last_update timestamptz; +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'; +SELECT id FROM users WHERE id = 1; + id +---- + 1 +(1 row) + +SELECT id FROM users WHERE id = 6; + id +---- + 6 +(1 row) + +ALTER TABLE items ADD COLUMN last_update timestamptz; +ROLLBACK; +-- 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; + id +---- + 1 +(1 row) + +SELECT id FROM users JOIN items ON (id = user_id) WHERE id = 6; + id +---- + 6 +(1 row) + +END; +BEGIN; +-- establish multiple connections to a node +\COPY users FROM STDIN WITH CSV +-- now read from the reference table over each connection +SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 2; + user_id +--------- +(0 rows) + +SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 3; + user_id +--------- +(0 rows) + +-- 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; +BEGIN; +-- establish multiple connections to a node +\COPY users FROM STDIN WITH CSV +-- read from the reference table over each connection +SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 2; + user_id +--------- +(0 rows) + +SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 3; + user_id +--------- +(0 rows) + +-- perform a DDL command on a co-located reference table +ALTER TABLE usergroups ADD COLUMN last_update timestamptz; +ERROR: cannot perform DDL on placement 1200034 since a co-located placement has been read over multiple connections +END; +BEGIN; +-- make a modification over connection 1 +INSERT INTO usergroups VALUES (0,'istanbul'); +-- copy over connections 1 and 2 +\COPY users FROM STDIN WITH CSV +-- cannot read modifications made over different connections +SELECT id FROM users JOIN usergroups ON (gid = user_group) WHERE id = 3; +ERROR: cannot perform query with placements that were modified over multiple connections +END; +-- make sure we can see cascading deletes +BEGIN; +DELETE FROM users; +SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 1; + user_id +--------- +(0 rows) + +SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 6; + user_id +--------- +(0 rows) + +END; +-- test visibility after COPY +INSERT INTO usergroups VALUES (2,'group'); +BEGIN; +-- opens two separate connections to node +\COPY users FROM STDIN WITH CSV +-- Uses first connection, which wrote the row with id = 2 +SELECT * FROM users JOIN usergroups ON (user_group = gid) WHERE id = 2; + id | name | user_group | gid | name +----+-------+------------+-----+------- + 2 | onder | 2 | 2 | group +(1 row) + +-- Should use second connection, which wrote the row with id = 4 +SELECT * FROM users JOIN usergroups ON (user_group = gid) WHERE id = 4; + id | name | user_group | gid | name +----+-------+------------+-----+------- + 4 | murat | 2 | 2 | group +(1 row) + +END; +-- make sure functions that throw an error roll back propertly +CREATE FUNCTION insert_abort() +RETURNS bool +AS $BODY$ +BEGIN + INSERT INTO labs VALUES (1001, 'Abort Labs'); + UPDATE labs SET name = 'Rollback Labs' WHERE id = 1001; + RAISE 'do not insert'; +END; +$BODY$ LANGUAGE plpgsql; +SELECT insert_abort(); +ERROR: do not insert +SELECT name FROM labs WHERE id = 1001; + name +------ +(0 rows) + +-- if function_opens_transaction-block is disabled the insert commits immediately +SET citus.function_opens_transaction_block TO off; +SELECT insert_abort(); +ERROR: do not insert +SELECT name FROM labs WHERE id = 1001; + name +--------------- + Rollback Labs +(1 row) + +RESET citus.function_opens_transaction_block; +DROP FUNCTION insert_abort(); +DROP TABLE items, users, itemgroups, usergroups, researchers, labs; diff --git a/src/test/regress/expected/multi_multiuser.out b/src/test/regress/expected/multi_multiuser.out index f7d046afb..966a0999f 100644 --- a/src/test/regress/expected/multi_multiuser.out +++ b/src/test/regress/expected/multi_multiuser.out @@ -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'); diff --git a/src/test/regress/expected/multi_multiuser_0.out b/src/test/regress/expected/multi_multiuser_0.out index 62314cc5b..32723752c 100644 --- a/src/test/regress/expected/multi_multiuser_0.out +++ b/src/test/regress/expected/multi_multiuser_0.out @@ -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'); diff --git a/src/test/regress/expected/multi_mx_explain.out b/src/test/regress/expected/multi_mx_explain.out index f7588299a..894903b1a 100644 --- a/src/test/regress/expected/multi_mx_explain.out +++ b/src/test/regress/expected/multi_mx_explain.out @@ -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) Custom Scan Outer - Citus Real-Time + Citus Adaptive false @@ -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 diff --git a/src/test/regress/expected/multi_mx_partitioning.out b/src/test/regress/expected/multi_mx_partitioning.out index 0cb0aa89b..a5427b6e2 100644 --- a/src/test/regress/expected/multi_mx_partitioning.out +++ b/src/test/regress/expected/multi_mx_partitioning.out @@ -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 diff --git a/src/test/regress/expected/multi_mx_partitioning_0.out b/src/test/regress/expected/multi_mx_partitioning_0.out index efdc0f28b..35bde35d1 100644 --- a/src/test/regress/expected/multi_mx_partitioning_0.out +++ b/src/test/regress/expected/multi_mx_partitioning_0.out @@ -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... ^ diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index bddacc469..0bfd18310 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -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 diff --git a/src/test/regress/expected/multi_null_minmax_value_pruning.out b/src/test/regress/expected/multi_null_minmax_value_pruning.out index 8b1b5b11e..e4fd0b758 100644 --- a/src/test/regress/expected/multi_null_minmax_value_pruning.out +++ b/src/test/regress/expected/multi_null_minmax_value_pruning.out @@ -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; diff --git a/src/test/regress/expected/multi_null_minmax_value_pruning_0.out b/src/test/regress/expected/multi_null_minmax_value_pruning_0.out index cbdb0a8d4..896e5747f 100644 --- a/src/test/regress/expected/multi_null_minmax_value_pruning_0.out +++ b/src/test/regress/expected/multi_null_minmax_value_pruning_0.out @@ -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 diff --git a/src/test/regress/expected/multi_partitioning.out b/src/test/regress/expected/multi_partitioning.out index ad5923d40..7d8e57c6f 100644 --- a/src/test/regress/expected/multi_partitioning.out +++ b/src/test/regress/expected/multi_partitioning.out @@ -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'); diff --git a/src/test/regress/expected/multi_partitioning_0.out b/src/test/regress/expected/multi_partitioning_0.out index 4522b321f..51a76ac90 100644 --- a/src/test/regress/expected/multi_partitioning_0.out +++ b/src/test/regress/expected/multi_partitioning_0.out @@ -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 diff --git a/src/test/regress/expected/multi_partitioning_9.out b/src/test/regress/expected/multi_partitioning_9.out new file mode 100644 index 000000000..6f46bd6f9 --- /dev/null +++ b/src/test/regress/expected/multi_partitioning_9.out @@ -0,0 +1,1939 @@ +-- +-- Distributed Partitioned Table Tests +-- +SET citus.next_shard_id TO 1660000; +SET citus.shard_count TO 4; +SET citus.shard_replication_factor TO 1; +-- print major version number for version-specific tests +SHOW server_version \gset +SELECT substring(:'server_version', '\d+')::int AS server_version; + server_version +---------------- + 11 +(1 row) + +-- +-- Distributed Partitioned Table Creation Tests +-- +-- 1-) Distributing partitioned table +-- create partitioned table +CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); +CREATE TABLE partitioning_hash_test(id int, subid int) PARTITION BY HASH(subid); +-- create its partitions +CREATE TABLE partitioning_test_2009 PARTITION OF partitioning_test FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +CREATE TABLE partitioning_test_2010 PARTITION OF partitioning_test FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +CREATE TABLE partitioning_hash_test_0 PARTITION OF partitioning_hash_test FOR VALUES WITH (MODULUS 3, REMAINDER 0); +CREATE TABLE partitioning_hash_test_1 PARTITION OF partitioning_hash_test FOR VALUES WITH (MODULUS 3, REMAINDER 1); +-- load some data and distribute tables +INSERT INTO partitioning_test VALUES (1, '2009-06-06'); +INSERT INTO partitioning_test VALUES (2, '2010-07-07'); +INSERT INTO partitioning_test_2009 VALUES (3, '2009-09-09'); +INSERT INTO partitioning_test_2010 VALUES (4, '2010-03-03'); +INSERT INTO partitioning_hash_test VALUES (1, 2); +INSERT INTO partitioning_hash_test VALUES (2, 13); +INSERT INTO partitioning_hash_test VALUES (3, 7); +INSERT INTO partitioning_hash_test VALUES (4, 4); +-- distribute partitioned table +SELECT create_distributed_table('partitioning_test', 'id'); +NOTICE: Copying data from local table... +NOTICE: Copying data from local table... + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('partitioning_hash_test', 'id'); +NOTICE: Copying data from local table... +NOTICE: Copying data from local table... + create_distributed_table +-------------------------- + +(1 row) + +-- see the data is loaded to shards +SELECT * FROM partitioning_test ORDER BY 1; + id | time +----+------------ + 1 | 06-06-2009 + 2 | 07-07-2010 + 3 | 09-09-2009 + 4 | 03-03-2010 +(4 rows) + +SELECT * FROM partitioning_hash_test ORDER BY 1; + id | subid +----+------- + 1 | 2 + 2 | 13 + 3 | 7 + 4 | 4 +(4 rows) + +-- see partitioned table and its partitions are distributed +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_test', 'partitioning_test_2009', 'partitioning_test_2010') +ORDER BY 1; + logicalrelid +------------------------ + partitioning_test + partitioning_test_2009 + partitioning_test_2010 +(3 rows) + +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2009', 'partitioning_test_2010') +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +------------------------+------- + partitioning_test | 4 + partitioning_test_2009 | 4 + partitioning_test_2010 | 4 +(3 rows) + +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_hash_test', 'partitioning_hash_test_0', 'partitioning_hash_test_1') +ORDER BY 1; + logicalrelid +-------------------------- + partitioning_hash_test + partitioning_hash_test_0 + partitioning_hash_test_1 +(3 rows) + +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_hash_test', 'partitioning_hash_test_0', 'partitioning_hash_test_1') +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +--------------------------+------- + partitioning_hash_test | 4 + partitioning_hash_test_0 | 4 + partitioning_hash_test_1 | 4 +(3 rows) + +-- 2-) Creating partition of a distributed table +CREATE TABLE partitioning_test_2011 PARTITION OF partitioning_test FOR VALUES FROM ('2011-01-01') TO ('2012-01-01'); +-- new partition is automatically distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_test', 'partitioning_test_2011') +ORDER BY 1; + logicalrelid +------------------------ + partitioning_test + partitioning_test_2011 +(2 rows) + +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2011') +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +------------------------+------- + partitioning_test | 4 + partitioning_test_2011 | 4 +(2 rows) + +-- 3-) Attaching non distributed table to a distributed table +CREATE TABLE partitioning_test_2012(id int, time date); +-- load some data +INSERT INTO partitioning_test_2012 VALUES (5, '2012-06-06'); +INSERT INTO partitioning_test_2012 VALUES (6, '2012-07-07'); +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2012 FOR VALUES FROM ('2012-01-01') TO ('2013-01-01'); +NOTICE: Copying data from local table... +-- attached partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_test', 'partitioning_test_2012') +ORDER BY 1; + logicalrelid +------------------------ + partitioning_test + partitioning_test_2012 +(2 rows) + +SELECT + logicalrelid, count(*) +FROM pg_dist_shard + WHERE logicalrelid IN ('partitioning_test', 'partitioning_test_2012') +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +------------------------+------- + partitioning_test | 4 + partitioning_test_2012 | 4 +(2 rows) + +-- try to insert a new data to hash partitioned table +-- no partition is defined for value 5 +INSERT INTO partitioning_hash_test VALUES (8, 5); +ERROR: no partition of relation "partitioning_hash_test_1660012" found for row +DETAIL: Partition key of the failing row contains (subid) = (5). +CONTEXT: while executing command on localhost:57637 +INSERT INTO partitioning_hash_test VALUES (9, 12); +ERROR: no partition of relation "partitioning_hash_test_1660015" found for row +DETAIL: Partition key of the failing row contains (subid) = (12). +CONTEXT: while executing command on localhost:57638 +CREATE TABLE partitioning_hash_test_2 (id int, subid int); +INSERT INTO partitioning_hash_test_2 VALUES (8, 5); +ALTER TABLE partitioning_hash_test ATTACH PARTITION partitioning_hash_test_2 FOR VALUES WITH (MODULUS 3, REMAINDER 2); +NOTICE: Copying data from local table... +INSERT INTO partitioning_hash_test VALUES (9, 12); +-- see the data is loaded to shards +SELECT * FROM partitioning_test ORDER BY 1; + id | time +----+------------ + 1 | 06-06-2009 + 2 | 07-07-2010 + 3 | 09-09-2009 + 4 | 03-03-2010 + 5 | 06-06-2012 + 6 | 07-07-2012 +(6 rows) + +SELECT * FROM partitioning_hash_test ORDER BY 1; + id | subid +----+------- + 1 | 2 + 2 | 13 + 3 | 7 + 4 | 4 + 8 | 5 + 9 | 12 +(6 rows) + +-- 4-) Attaching distributed table to distributed table +CREATE TABLE partitioning_test_2013(id int, time date); +SELECT create_distributed_table('partitioning_test_2013', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +-- load some data +INSERT INTO partitioning_test_2013 VALUES (7, '2013-06-06'); +INSERT INTO partitioning_test_2013 VALUES (8, '2013-07-07'); +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2013 FOR VALUES FROM ('2013-01-01') TO ('2014-01-01'); +-- see the data is loaded to shards +SELECT * FROM partitioning_test ORDER BY 1; + id | time +----+------------ + 1 | 06-06-2009 + 2 | 07-07-2010 + 3 | 09-09-2009 + 4 | 03-03-2010 + 5 | 06-06-2012 + 6 | 07-07-2012 + 7 | 06-06-2013 + 8 | 07-07-2013 +(8 rows) + +-- 5-) Failure cases while creating distributed partitioned tables +-- cannot distribute a partition if its parent is not distributed +CREATE TABLE partitioning_test_failure(id int, time date) PARTITION BY RANGE (time); +CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); +ERROR: cannot distribute relation "partitioning_test_failure_2009" which is partition of "partitioning_test_failure" +DETAIL: Citus does not support distributing partitions if their parent is not distributed table. +HINT: Distribute the partitioned table "partitioning_test_failure" instead. +-- only hash distributed tables can have partitions +SELECT create_distributed_table('partitioning_test_failure', 'id', 'append'); +ERROR: distributing partitioned tables in only supported for hash-distributed tables +SELECT create_distributed_table('partitioning_test_failure', 'id', 'range'); +ERROR: distributing partitioned tables in only supported for hash-distributed tables +SELECT create_reference_table('partitioning_test_failure'); +ERROR: distributing partitioned tables in only supported for hash-distributed tables +SET citus.shard_replication_factor TO 1; +-- non-distributed tables cannot have distributed partitions; +DROP TABLE partitioning_test_failure_2009; +CREATE TABLE partitioning_test_failure_2009(id int, time date); +SELECT create_distributed_table('partitioning_test_failure_2009', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE partitioning_test_failure ATTACH PARTITION partitioning_test_failure_2009 FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +ERROR: non-distributed tables cannot have distributed partitions +HINT: Distribute the partitioned table "partitioning_test_failure_2009" instead +-- multi-level partitioning is not allowed +DROP TABLE partitioning_test_failure_2009; +CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time); +SELECT create_distributed_table('partitioning_test_failure', 'id'); +ERROR: distributing multi-level partitioned tables is not supported +DETAIL: Relation "partitioning_test_failure_2009" is partitioned table itself and it is also partition of relation "partitioning_test_failure". +-- multi-level partitioning is not allowed in different order +DROP TABLE partitioning_test_failure_2009; +SELECT create_distributed_table('partitioning_test_failure', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE partitioning_test_failure_2009 PARTITION OF partitioning_test_failure FOR VALUES FROM ('2009-01-01') TO ('2010-01-01') PARTITION BY RANGE (time); +ERROR: distributing multi-level partitioned tables is not supported +DETAIL: Relation "partitioning_test_failure_2009" is partitioned table itself and it is also partition of relation "partitioning_test_failure". +-- +-- DMLs in distributed partitioned tables +-- +-- test COPY +-- COPY data to partitioned table +COPY partitioning_test FROM STDIN WITH CSV; +-- COPY data to partition directly +COPY partitioning_test_2009 FROM STDIN WITH CSV; +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id >= 9 ORDER BY 1; + id | time +----+------------ + 9 | 01-01-2009 + 10 | 01-01-2010 + 11 | 01-01-2011 + 12 | 01-01-2012 + 13 | 01-02-2009 + 14 | 01-03-2009 +(6 rows) + +-- test INSERT +-- INSERT INTO the partitioned table +INSERT INTO partitioning_test VALUES(15, '2009-02-01'); +INSERT INTO partitioning_test VALUES(16, '2010-02-01'); +INSERT INTO partitioning_test VALUES(17, '2011-02-01'); +INSERT INTO partitioning_test VALUES(18, '2012-02-01'); +-- INSERT INTO the partitions directly table +INSERT INTO partitioning_test VALUES(19, '2009-02-02'); +INSERT INTO partitioning_test VALUES(20, '2010-02-02'); +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id >= 15 ORDER BY 1; + id | time +----+------------ + 15 | 02-01-2009 + 16 | 02-01-2010 + 17 | 02-01-2011 + 18 | 02-01-2012 + 19 | 02-02-2009 + 20 | 02-02-2010 +(6 rows) + +-- test INSERT/SELECT +-- INSERT/SELECT from partition to partitioned table +INSERT INTO partitioning_test SELECT * FROM partitioning_test_2011; +-- INSERT/SELECT from partitioned table to partition +INSERT INTO partitioning_test_2012 SELECT * FROM partitioning_test WHERE time >= '2012-01-01' AND time < '2013-01-01'; +-- see the data is loaded to shards (rows in the given range should be duplicated) +SELECT * FROM partitioning_test WHERE time >= '2011-01-01' AND time < '2013-01-01' ORDER BY 1; + id | time +----+------------ + 5 | 06-06-2012 + 5 | 06-06-2012 + 6 | 07-07-2012 + 6 | 07-07-2012 + 11 | 01-01-2011 + 11 | 01-01-2011 + 12 | 01-01-2012 + 12 | 01-01-2012 + 17 | 02-01-2011 + 17 | 02-01-2011 + 18 | 02-01-2012 + 18 | 02-01-2012 +(12 rows) + +-- test UPDATE +-- UPDATE partitioned table +UPDATE partitioning_test SET time = '2013-07-07' WHERE id = 7; +-- UPDATE partition directly +UPDATE partitioning_test_2013 SET time = '2013-08-08' WHERE id = 8; +-- see the data is updated +SELECT * FROM partitioning_test WHERE id = 7 OR id = 8 ORDER BY 1; + id | time +----+------------ + 7 | 07-07-2013 + 8 | 08-08-2013 +(2 rows) + +-- UPDATE that tries to move a row to a non-existing partition (this should fail) +UPDATE partitioning_test SET time = '2020-07-07' WHERE id = 7; +ERROR: no partition of relation "partitioning_test_1660001" found for row +DETAIL: Partition key of the failing row contains ("time") = (2020-07-07). +CONTEXT: while executing command on localhost:57638 +-- UPDATE with subqueries on partitioned table +UPDATE + partitioning_test +SET + time = time + INTERVAL '1 day' +WHERE + id IN (SELECT id FROM partitioning_test WHERE id = 1); +-- UPDATE with subqueries on partition +UPDATE + partitioning_test_2009 +SET + time = time + INTERVAL '1 month' +WHERE + id IN (SELECT id FROM partitioning_test WHERE id = 2); +-- see the data is updated +SELECT * FROM partitioning_test WHERE id = 1 OR id = 2 ORDER BY 1; + id | time +----+------------ + 1 | 06-07-2009 + 2 | 07-07-2010 +(2 rows) + +-- test DELETE +-- DELETE from partitioned table +DELETE FROM partitioning_test WHERE id = 9; +-- DELETE from partition directly +DELETE FROM partitioning_test_2010 WHERE id = 10; +-- see the data is deleted +SELECT * FROM partitioning_test WHERE id = 9 OR id = 10 ORDER BY 1; + id | time +----+------ +(0 rows) + +-- create default partition +CREATE TABLE partitioning_test_default PARTITION OF partitioning_test DEFAULT; +\d+ partitioning_test + Table "public.partitioning_test" + Column | Type | Collation | Nullable | Default | Storage | Stats target | Description +--------+---------+-----------+----------+---------+---------+--------------+------------- + id | integer | | | | plain | | + time | date | | | | plain | | +Partition key: RANGE ("time") +Partitions: partitioning_test_2009 FOR VALUES FROM ('01-01-2009') TO ('01-01-2010'), + partitioning_test_2010 FOR VALUES FROM ('01-01-2010') TO ('01-01-2011'), + partitioning_test_2011 FOR VALUES FROM ('01-01-2011') TO ('01-01-2012'), + partitioning_test_2012 FOR VALUES FROM ('01-01-2012') TO ('01-01-2013'), + partitioning_test_2013 FOR VALUES FROM ('01-01-2013') TO ('01-01-2014'), + partitioning_test_default DEFAULT + +INSERT INTO partitioning_test VALUES(21, '2014-02-02'); +INSERT INTO partitioning_test VALUES(22, '2015-04-02'); +-- see they are inserted into default partition +SELECT * FROM partitioning_test WHERE id > 20 ORDER BY 1, 2; + id | time +----+------------ + 21 | 02-02-2014 + 22 | 04-02-2015 +(2 rows) + +SELECT * FROM partitioning_test_default ORDER BY 1, 2; + id | time +----+------------ + 21 | 02-02-2014 + 22 | 04-02-2015 +(2 rows) + +-- create a new partition (will fail) +CREATE TABLE partitioning_test_2014 PARTITION OF partitioning_test FOR VALUES FROM ('2014-01-01') TO ('2015-01-01'); +ERROR: updated partition constraint for default partition would be violated by some row +CONTEXT: while executing command on localhost:57637 +BEGIN; +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_default; +CREATE TABLE partitioning_test_2014 PARTITION OF partitioning_test FOR VALUES FROM ('2014-01-01') TO ('2015-01-01'); +INSERT INTO partitioning_test SELECT * FROM partitioning_test_default WHERE time >= '2014-01-01' AND time < '2015-01-01'; +DELETE FROM partitioning_test_default WHERE time >= '2014-01-01' AND time < '2015-01-01'; +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_default DEFAULT; +END; +-- see data is in the table, but some moved out from default partition +SELECT * FROM partitioning_test WHERE id > 20 ORDER BY 1, 2; + id | time +----+------------ + 21 | 02-02-2014 + 22 | 04-02-2015 +(2 rows) + +SELECT * FROM partitioning_test_default ORDER BY 1, 2; + id | time +----+------------ + 22 | 04-02-2015 +(1 row) + +-- multi-shard UPDATE on partitioned table +UPDATE partitioning_test SET time = time + INTERVAL '1 day'; +-- see rows are UPDATED +SELECT * FROM partitioning_test ORDER BY 1; + id | time +----+------------ + 1 | 06-08-2009 + 2 | 07-08-2010 + 3 | 09-10-2009 + 4 | 03-04-2010 + 5 | 06-07-2012 + 5 | 06-07-2012 + 6 | 07-08-2012 + 6 | 07-08-2012 + 7 | 07-08-2013 + 8 | 08-09-2013 + 11 | 01-02-2011 + 11 | 01-02-2011 + 12 | 01-02-2012 + 12 | 01-02-2012 + 13 | 01-03-2009 + 14 | 01-04-2009 + 15 | 02-02-2009 + 16 | 02-02-2010 + 17 | 02-02-2011 + 17 | 02-02-2011 + 18 | 02-02-2012 + 18 | 02-02-2012 + 19 | 02-03-2009 + 20 | 02-03-2010 + 21 | 02-03-2014 + 22 | 04-03-2015 +(26 rows) + +-- multi-shard UPDATE on partition directly +UPDATE partitioning_test_2009 SET time = time + INTERVAL '1 day'; +-- see rows are UPDATED +SELECT * FROM partitioning_test_2009 ORDER BY 1; + id | time +----+------------ + 1 | 06-09-2009 + 3 | 09-11-2009 + 13 | 01-04-2009 + 14 | 01-05-2009 + 15 | 02-03-2009 + 19 | 02-04-2009 +(6 rows) + +-- test multi-shard UPDATE which fails in workers (updated value is outside of partition bounds) +UPDATE partitioning_test_2009 SET time = time + INTERVAL '6 month'; +ERROR: new row for relation "partitioning_test_2009_1660005" violates partition constraint +DETAIL: Failing row contains (3, 2010-03-11). +CONTEXT: while executing command on localhost:57638 +-- +-- DDL in distributed partitioned tables +-- +-- test CREATE INDEX +-- CREATE INDEX on partitioned table - this will error out +-- on earlier versions of postgres earlier than 11. +CREATE INDEX partitioning_index ON partitioning_test(id); +-- CREATE INDEX on partition +CREATE INDEX partitioning_2009_index ON partitioning_test_2009(id); +-- CREATE INDEX CONCURRENTLY on partition +CREATE INDEX CONCURRENTLY partitioned_2010_index ON partitioning_test_2010(id); +-- see index is created +SELECT tablename, indexname FROM pg_indexes WHERE tablename LIKE 'partitioning_test%' ORDER BY indexname; + tablename | indexname +---------------------------+---------------------------------- + partitioning_test_2010 | partitioned_2010_index + partitioning_test_2009 | partitioning_2009_index + partitioning_test_2009 | partitioning_test_2009_id_idx + partitioning_test_2010 | partitioning_test_2010_id_idx + partitioning_test_2011 | partitioning_test_2011_id_idx + partitioning_test_2012 | partitioning_test_2012_id_idx + partitioning_test_2013 | partitioning_test_2013_id_idx + partitioning_test_2014 | partitioning_test_2014_id_idx + partitioning_test_default | partitioning_test_default_id_idx +(9 rows) + +-- test drop +-- indexes created on parent table can only be dropped on parent table +-- ie using the same index name +-- following will fail +DROP INDEX partitioning_test_2009_id_idx; +ERROR: cannot drop index partitioning_test_2009_id_idx because index partitioning_index requires it +HINT: You can drop index partitioning_index instead. +-- but dropping index on parent table will succeed +DROP INDEX partitioning_index; +-- this index was already created on partition table +DROP INDEX partitioning_2009_index; +-- test drop index on non-distributed, partitioned table +CREATE TABLE non_distributed_partitioned_table(a int, b int) PARTITION BY RANGE (a); +CREATE TABLE non_distributed_partitioned_table_1 PARTITION OF non_distributed_partitioned_table +FOR VALUES FROM (0) TO (10); +CREATE INDEX non_distributed_partitioned_table_index ON non_distributed_partitioned_table(a); +-- see index is created +SELECT tablename, indexname FROM pg_indexes WHERE tablename LIKE 'non_distributed%' ORDER BY indexname; + tablename | indexname +-------------------------------------+------------------------------------------- + non_distributed_partitioned_table_1 | non_distributed_partitioned_table_1_a_idx +(1 row) + +-- drop the index and see it is dropped +DROP INDEX non_distributed_partitioned_table_index; +SELECT tablename, indexname FROM pg_indexes WHERE tablename LIKE 'non_distributed%' ORDER BY indexname; + tablename | indexname +-----------+----------- +(0 rows) + +-- test add COLUMN +-- add COLUMN to partitioned table +ALTER TABLE partitioning_test ADD new_column int; +-- add COLUMN to partition - this will error out +ALTER TABLE partitioning_test_2010 ADD new_column_2 int; +ERROR: cannot add column to a partition +-- see additional column is created +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; + name | type +------------+--------- + id | integer + new_column | integer + time | date +(3 rows) + +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test_2010'::regclass ORDER BY 1; + name | type +------------+--------- + id | integer + new_column | integer + time | date +(3 rows) + +-- test add PRIMARY KEY +-- add PRIMARY KEY to partitioned table - this will error out +ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_primary PRIMARY KEY (id); +ERROR: insufficient columns in PRIMARY KEY constraint definition +DETAIL: PRIMARY KEY constraint on table "partitioning_test" lacks column "time" which is part of the partition key. +-- ADD PRIMARY KEY to partition +ALTER TABLE partitioning_test_2009 ADD CONSTRAINT partitioning_2009_primary PRIMARY KEY (id); +-- see PRIMARY KEY is created +SELECT + table_name, + constraint_name, + constraint_type +FROM + information_schema.table_constraints +WHERE + table_name = 'partitioning_test_2009' AND + constraint_name = 'partitioning_2009_primary'; + table_name | constraint_name | constraint_type +------------------------+---------------------------+----------------- + partitioning_test_2009 | partitioning_2009_primary | PRIMARY KEY +(1 row) + +-- however, you can add primary key if it contains both distribution and partition key +ALTER TABLE partitioning_hash_test ADD CONSTRAINT partitioning_hash_primary PRIMARY KEY (id, subid); +-- see PRIMARY KEY is created +SELECT + table_name, + constraint_name, + constraint_type +FROM + information_schema.table_constraints +WHERE + table_name LIKE 'partitioning_hash_test%' AND + constraint_type = 'PRIMARY KEY' +ORDER BY 1; + table_name | constraint_name | constraint_type +--------------------------+-------------------------------+----------------- + partitioning_hash_test | partitioning_hash_primary | PRIMARY KEY + partitioning_hash_test_0 | partitioning_hash_test_0_pkey | PRIMARY KEY + partitioning_hash_test_1 | partitioning_hash_test_1_pkey | PRIMARY KEY + partitioning_hash_test_2 | partitioning_hash_test_2_pkey | PRIMARY KEY +(4 rows) + +-- test ADD FOREIGN CONSTRAINT +-- add FOREIGN CONSTRAINT to partitioned table -- this will error out (it is a self reference) +ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id); +ERROR: cannot ALTER TABLE "partitioning_test_2009" because it is being used by active queries in this session +-- add FOREIGN CONSTRAINT to partition +INSERT INTO partitioning_test_2009 VALUES (5, '2009-06-06'); +INSERT INTO partitioning_test_2009 VALUES (6, '2009-07-07'); +INSERT INTO partitioning_test_2009 VALUES(12, '2009-02-01'); +INSERT INTO partitioning_test_2009 VALUES(18, '2009-02-01'); +ALTER TABLE partitioning_test_2012 ADD CONSTRAINT partitioning_2012_foreign FOREIGN KEY (id) REFERENCES partitioning_test_2009 (id) ON DELETE CASCADE; +-- see FOREIGN KEY is created +SELECT "Constraint" FROM table_fkeys WHERE relid = 'partitioning_test_2012'::regclass ORDER BY 1; + Constraint +--------------------------- + partitioning_2012_foreign +(1 row) + +-- test ON DELETE CASCADE works +DELETE FROM partitioning_test_2009 WHERE id = 5; +-- see that element is deleted from both partitions +SELECT * FROM partitioning_test_2009 WHERE id = 5 ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +SELECT * FROM partitioning_test_2012 WHERE id = 5 ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- test DETACH partition +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009; +-- see DETACHed partitions content is not accessible from partitioning_test; +SELECT * FROM partitioning_test WHERE time >= '2009-01-01' AND time < '2010-01-01' ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- delete from default partition +DELETE FROM partitioning_test WHERE time >= '2015-01-01'; +SELECT * FROM partitioning_test_default; + id | time | new_column +----+------+------------ +(0 rows) + +-- create a reference table for foreign key test +CREATE TABLE partitioning_test_reference(id int PRIMARY KEY, subid int); +INSERT INTO partitioning_test_reference SELECT a, a FROM generate_series(1, 50) a; +SELECT create_reference_table('partitioning_test_reference'); +NOTICE: Copying data from local table... + create_reference_table +------------------------ + +(1 row) + +ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_reference_fkey FOREIGN KEY (id) REFERENCES partitioning_test_reference(id) ON DELETE CASCADE; +CREATE TABLE partitioning_test_foreign_key(id int PRIMARY KEY, value int); +SELECT create_distributed_table('partitioning_test_foreign_key', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO partitioning_test_foreign_key SELECT * FROM partitioning_test_reference; +ALTER TABLE partitioning_hash_test ADD CONSTRAINT partitioning_reference_fk_test FOREIGN KEY (id) REFERENCES partitioning_test_foreign_key(id) ON DELETE CASCADE; +-- check foreign keys on partitions +SELECT + table_name, constraint_name, constraint_type FROm information_schema.table_constraints +WHERE + table_name LIKE 'partitioning_hash_test%' AND + constraint_type = 'FOREIGN KEY' +ORDER BY + 1,2; + table_name | constraint_name | constraint_type +--------------------------+--------------------------------+----------------- + partitioning_hash_test | partitioning_reference_fk_test | FOREIGN KEY + partitioning_hash_test_0 | partitioning_reference_fk_test | FOREIGN KEY + partitioning_hash_test_1 | partitioning_reference_fk_test | FOREIGN KEY + partitioning_hash_test_2 | partitioning_reference_fk_test | FOREIGN KEY +(4 rows) + +-- check foreign keys on partition shards +-- there is some text ordering issue regarding table name +-- forcing integer sort by extracting shardid +CREATE TYPE foreign_key_details AS (table_name text, constraint_name text, constraint_type text); +SELECT right(table_name, 7)::int as shardid, * FROM ( + SELECT (json_populate_record(NULL::foreign_key_details, + json_array_elements_text(result::json)::json )).* + FROM run_command_on_workers($$ + SELECT + COALESCE(json_agg(row_to_json(q)), '[]'::json) + FROM ( + SELECT + table_name, constraint_name, constraint_type + FROM information_schema.table_constraints + WHERE + table_name LIKE 'partitioning_hash_test%' AND + constraint_type = 'FOREIGN KEY' + ORDER BY 1, 2, 3 + ) q + $$) ) w +ORDER BY 1, 2, 3, 4; + shardid | table_name | constraint_name | constraint_type +---------+----------------------------------+----------------------------------------+----------------- + 1660012 | partitioning_hash_test_1660012 | partitioning_reference_fk_test_1660012 | FOREIGN KEY + 1660013 | partitioning_hash_test_1660013 | partitioning_reference_fk_test_1660013 | FOREIGN KEY + 1660014 | partitioning_hash_test_1660014 | partitioning_reference_fk_test_1660014 | FOREIGN KEY + 1660015 | partitioning_hash_test_1660015 | partitioning_reference_fk_test_1660015 | FOREIGN KEY + 1660016 | partitioning_hash_test_0_1660016 | partitioning_reference_fk_test_1660012 | FOREIGN KEY + 1660017 | partitioning_hash_test_0_1660017 | partitioning_reference_fk_test_1660013 | FOREIGN KEY + 1660018 | partitioning_hash_test_0_1660018 | partitioning_reference_fk_test_1660014 | FOREIGN KEY + 1660019 | partitioning_hash_test_0_1660019 | partitioning_reference_fk_test_1660015 | FOREIGN KEY + 1660020 | partitioning_hash_test_1_1660020 | partitioning_reference_fk_test_1660012 | FOREIGN KEY + 1660021 | partitioning_hash_test_1_1660021 | partitioning_reference_fk_test_1660013 | FOREIGN KEY + 1660022 | partitioning_hash_test_1_1660022 | partitioning_reference_fk_test_1660014 | FOREIGN KEY + 1660023 | partitioning_hash_test_1_1660023 | partitioning_reference_fk_test_1660015 | FOREIGN KEY + 1660032 | partitioning_hash_test_2_1660032 | partitioning_reference_fk_test_1660012 | FOREIGN KEY + 1660033 | partitioning_hash_test_2_1660033 | partitioning_reference_fk_test_1660013 | FOREIGN KEY + 1660034 | partitioning_hash_test_2_1660034 | partitioning_reference_fk_test_1660014 | FOREIGN KEY + 1660035 | partitioning_hash_test_2_1660035 | partitioning_reference_fk_test_1660015 | FOREIGN KEY +(16 rows) + +DROP TYPE foreign_key_details; +-- set replication factor back to 1 since it gots reset +-- after connection re-establishment +SET citus.shard_replication_factor TO 1; +SELECT * FROM partitioning_test WHERE id = 11 or id = 12; + id | time | new_column +----+------------+------------ + 11 | 01-02-2011 | + 11 | 01-02-2011 | + 12 | 01-02-2012 | + 12 | 01-02-2012 | +(4 rows) + +DELETE FROM partitioning_test_reference WHERE id = 11 or id = 12; +SELECT * FROM partitioning_hash_test ORDER BY 1, 2; + id | subid +----+------- + 1 | 2 + 2 | 13 + 3 | 7 + 4 | 4 + 8 | 5 + 9 | 12 +(6 rows) + +DELETE FROM partitioning_test_foreign_key WHERE id = 2 OR id = 9; +-- see data is deleted from referencing table +SELECT * FROM partitioning_test WHERE id = 11 or id = 12; + id | time | new_column +----+------+------------ +(0 rows) + +SELECT * FROM partitioning_hash_test ORDER BY 1, 2; + id | subid +----+------- + 1 | 2 + 3 | 7 + 4 | 4 + 8 | 5 +(4 rows) + +-- +-- Transaction tests +-- +-- DDL in transaction +BEGIN; +ALTER TABLE partitioning_test ADD newer_column int; +-- see additional column is created +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; + name | type +--------------+--------- + id | integer + new_column | integer + newer_column | integer + time | date +(4 rows) + +ROLLBACK; +-- see rollback is successful +SELECT name, type FROM table_attrs WHERE relid = 'partitioning_test'::regclass ORDER BY 1; + name | type +------------+--------- + id | integer + new_column | integer + time | date +(3 rows) + +-- COPY in transaction +BEGIN; +COPY partitioning_test FROM STDIN WITH CSV; +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id = 22 ORDER BY 1; + id | time | new_column +----+------------+------------ + 22 | 01-01-2010 | 22 +(1 row) + +SELECT * FROM partitioning_test WHERE id = 23 ORDER BY 1; + id | time | new_column +----+------------+------------ + 23 | 01-01-2011 | 23 +(1 row) + +SELECT * FROM partitioning_test WHERE id = 24 ORDER BY 1; + id | time | new_column +----+------------+------------ + 24 | 01-01-2013 | 24 +(1 row) + +ROLLBACK; +-- see rollback is successful +SELECT * FROM partitioning_test WHERE id >= 22 ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- DML in transaction +BEGIN; +-- INSERT in transaction +INSERT INTO partitioning_test VALUES(25, '2010-02-02'); +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; + id | time | new_column +----+------------+------------ + 25 | 02-02-2010 | +(1 row) + +-- INSERT/SELECT in transaction +INSERT INTO partitioning_test SELECT * FROM partitioning_test WHERE id = 25; +-- see the data is loaded to shards +SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; + id | time | new_column +----+------------+------------ + 25 | 02-02-2010 | + 25 | 02-02-2010 | +(2 rows) + +-- UPDATE in transaction +UPDATE partitioning_test SET time = '2010-10-10' WHERE id = 25; +-- see the data is updated +SELECT * FROM partitioning_test WHERE id = 25 ORDER BY 1; + id | time | new_column +----+------------+------------ + 25 | 10-10-2010 | + 25 | 10-10-2010 | +(2 rows) + +-- perform operations on partition and partioned tables together +INSERT INTO partitioning_test VALUES(26, '2010-02-02', 26); +INSERT INTO partitioning_test_2010 VALUES(26, '2010-02-02', 26); +COPY partitioning_test FROM STDIN WITH CSV; +COPY partitioning_test_2010 FROM STDIN WITH CSV; +-- see the data is loaded to shards (we should see 4 rows with same content) +SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; + id | time | new_column +----+------------+------------ + 26 | 02-02-2010 | 26 + 26 | 02-02-2010 | 26 + 26 | 02-02-2010 | 26 + 26 | 02-02-2010 | 26 +(4 rows) + +ROLLBACK; +-- see rollback is successful +SELECT * FROM partitioning_test WHERE id = 26 ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- DETACH and DROP in a transaction +BEGIN; +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2011; +DROP TABLE partitioning_test_2011; +COMMIT; +-- see DROPed partitions content is not accessible +SELECT * FROM partitioning_test WHERE time >= '2011-01-01' AND time < '2012-01-01' ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- +-- Misc tests +-- +-- test TRUNCATE +-- test TRUNCATE partition +TRUNCATE partitioning_test_2012; +-- see partition is TRUNCATEd +SELECT * FROM partitioning_test_2012 ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- test TRUNCATE partitioned table +TRUNCATE partitioning_test; +-- see partitioned table is TRUNCATEd +SELECT * FROM partitioning_test ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- test DROP +-- test DROP partition +INSERT INTO partitioning_test_2010 VALUES(27, '2010-02-01'); +DROP TABLE partitioning_test_2010; +-- see DROPped partitions content is not accessible from partitioning_test; +SELECT * FROM partitioning_test WHERE time >= '2010-01-01' AND time < '2011-01-01' ORDER BY 1; + id | time | new_column +----+------+------------ +(0 rows) + +-- test DROP partitioned table +DROP TABLE partitioning_test; +DROP TABLE partitioning_test_reference; +-- dropping the parent should CASCADE to the children as well +SELECT table_name FROM information_schema.tables WHERE table_name LIKE 'partitioning_test%' ORDER BY 1; + table_name +------------------------------- + partitioning_test_2009 + partitioning_test_failure + partitioning_test_foreign_key +(3 rows) + +-- test distributing partitioned table colocated with non-partitioned table +CREATE TABLE partitioned_users_table (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint) PARTITION BY RANGE (time); +CREATE TABLE partitioned_events_table (user_id int, time timestamp, event_type int, value_2 int, value_3 float, value_4 bigint) PARTITION BY RANGE (time); +SELECT create_distributed_table('partitioned_users_table', 'user_id', colocate_with => 'users_table'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('partitioned_events_table', 'user_id', colocate_with => 'events_table'); + create_distributed_table +-------------------------- + +(1 row) + +-- INSERT/SELECT from regular table to partitioned table +CREATE TABLE partitioned_users_table_2009 PARTITION OF partitioned_users_table FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +CREATE TABLE partitioned_events_table_2009 PARTITION OF partitioned_events_table FOR VALUES FROM ('2017-01-01') TO ('2018-01-01'); +INSERT INTO partitioned_events_table SELECT * FROM events_table; +INSERT INTO partitioned_users_table_2009 SELECT * FROM users_table; +-- +-- Complex JOINs, subqueries, UNIONs etc... +-- +-- subquery with UNIONs on partitioned table +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + (SELECT *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + (SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM( + (SELECT + "events"."user_id", "events"."time", 0 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (1, 2) ) + UNION + (SELECT + "events"."user_id", "events"."time", 1 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (3, 4) ) + UNION + (SELECT + "events"."user_id", "events"."time", 2 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (5, 6) ) + UNION + (SELECT + "events"."user_id", "events"."time", 3 AS event + FROM + partitioned_events_table as "events" + WHERE + event_type IN (1, 6))) t1 + GROUP BY "t1"."user_id") AS t) "q" +) AS final_query +GROUP BY types +ORDER BY types; + types | sumofeventtype +-------+---------------- + 0 | 43 + 1 | 44 + 2 | 8 + 3 | 25 +(4 rows) + +-- UNION and JOIN on both partitioned and regular tables +SELECT ("final_query"."event_types") as types, count(*) AS sumOfEventType +FROM + (SELECT + *, random() + FROM + (SELECT + "t"."user_id", "t"."time", unnest("t"."collected_events") AS "event_types" + FROM + (SELECT + "t1"."user_id", min("t1"."time") AS "time", array_agg(("t1"."event") ORDER BY TIME ASC, event DESC) AS collected_events + FROM ( + (SELECT + * + FROM + (SELECT + "events"."time", 0 AS event, "events"."user_id" + FROM + partitioned_events_table as "events" + WHERE + event_type IN (1, 2)) events_subquery_1) + UNION + (SELECT * + FROM + ( + SELECT * FROM + ( + SELECT + max("events"."time"), + 0 AS event, + "events"."user_id" + FROM + events_table as "events", users_table as "users" + WHERE + events.user_id = users.user_id AND + event_type IN (1, 2) + GROUP BY "events"."user_id" + ) as events_subquery_5 + ) events_subquery_2) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 2 AS event, "events"."user_id" + FROM + partitioned_events_table as "events" + WHERE + event_type IN (3, 4)) events_subquery_3) + UNION + (SELECT * + FROM + (SELECT + "events"."time", 3 AS event, "events"."user_id" + FROM + events_table as "events" + WHERE + event_type IN (5, 6)) events_subquery_4) + ) t1 + GROUP BY "t1"."user_id") AS t) "q" +INNER JOIN + (SELECT + "users"."user_id" + FROM + partitioned_users_table as "users" + WHERE + value_1 > 2 and value_1 < 5) AS t + ON (t.user_id = q.user_id)) as final_query +GROUP BY + types +ORDER BY + types; + types | sumofeventtype +-------+---------------- + 0 | 367 + 2 | 360 + 3 | 57 +(3 rows) + +-- test LIST partitioning +CREATE TABLE list_partitioned_events_table (user_id int, time date, event_type int, value_2 int, value_3 float, value_4 bigint) PARTITION BY LIST (time); +CREATE TABLE list_partitioned_events_table_2014_01_01_05 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-11-21', '2017-11-22', '2017-11-23', '2017-11-24', '2017-11-25'); +CREATE TABLE list_partitioned_events_table_2014_01_06_10 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-11-26', '2017-11-27', '2017-11-28', '2017-11-29', '2017-11-30'); +CREATE TABLE list_partitioned_events_table_2014_01_11_15 PARTITION OF list_partitioned_events_table FOR VALUES IN ('2017-12-01', '2017-12-02', '2017-12-03', '2017-12-04', '2017-12-05'); +-- test distributing partitioned table colocated with another partitioned table +SELECT create_distributed_table('list_partitioned_events_table', 'user_id', colocate_with => 'partitioned_events_table'); + create_distributed_table +-------------------------- + +(1 row) + +-- INSERT/SELECT from partitioned table to partitioned table +INSERT INTO + list_partitioned_events_table +SELECT + user_id, + date_trunc('day', time) as time, + event_type, + value_2, + value_3, + value_4 +FROM + events_table +WHERE + time >= '2017-11-21' AND + time <= '2017-12-01'; +-- LEFT JOINs used with INNER JOINs on range partitioned table, list partitioned table and non-partitioned table +SELECT +count(*) AS cnt, "generated_group_field" + FROM + (SELECT + "eventQuery"."user_id", random(), generated_group_field + FROM + (SELECT + "multi_group_wrapper_1".*, generated_group_field, random() + FROM + (SELECT * + FROM + (SELECT + "list_partitioned_events_table"."time", "list_partitioned_events_table"."user_id" as event_user_id + FROM + list_partitioned_events_table as "list_partitioned_events_table" + WHERE + user_id > 2) "temp_data_queries" + INNER JOIN + (SELECT + "users"."user_id" + FROM + partitioned_users_table as "users" + WHERE + user_id > 2 and value_2 = 1) "user_filters_1" + ON ("temp_data_queries".event_user_id = "user_filters_1".user_id)) AS "multi_group_wrapper_1" + LEFT JOIN + (SELECT + "users"."user_id" AS "user_id", value_2 AS "generated_group_field" + FROM + partitioned_users_table as "users") "left_group_by_1" + ON ("left_group_by_1".user_id = "multi_group_wrapper_1".event_user_id)) "eventQuery") "pushedDownQuery" + GROUP BY + "generated_group_field" + ORDER BY + cnt DESC, generated_group_field ASC + LIMIT 10; + cnt | generated_group_field +------+----------------------- + 1851 | 1 + 1077 | 4 + 963 | 2 + 955 | 3 + 768 | 5 + 639 | 0 +(6 rows) + +-- +-- Additional partitioning features +-- +-- test multi column partitioning +CREATE TABLE multi_column_partitioning(c1 int, c2 int) PARTITION BY RANGE (c1, c2); +CREATE TABLE multi_column_partitioning_0_0_10_0 PARTITION OF multi_column_partitioning FOR VALUES FROM (0, 0) TO (10, 0); +SELECT create_distributed_table('multi_column_partitioning', 'c1'); + create_distributed_table +-------------------------- + +(1 row) + +-- test INSERT to multi-column partitioned table +INSERT INTO multi_column_partitioning VALUES(1, 1); +INSERT INTO multi_column_partitioning_0_0_10_0 VALUES(5, -5); +-- test INSERT to multi-column partitioned table where no suitable partition exists +INSERT INTO multi_column_partitioning VALUES(10, 1); +ERROR: no partition of relation "multi_column_partitioning_1660101" found for row +DETAIL: Partition key of the failing row contains (c1, c2) = (10, 1). +CONTEXT: while executing command on localhost:57637 +-- test with MINVALUE/MAXVALUE +CREATE TABLE multi_column_partitioning_10_max_20_min PARTITION OF multi_column_partitioning FOR VALUES FROM (10, MAXVALUE) TO (20, MINVALUE); +-- test INSERT to partition with MINVALUE/MAXVALUE bounds +INSERT INTO multi_column_partitioning VALUES(11, -11); +INSERT INTO multi_column_partitioning_10_max_20_min VALUES(19, -19); +-- test INSERT to multi-column partitioned table where no suitable partition exists +INSERT INTO multi_column_partitioning VALUES(20, -20); +ERROR: no partition of relation "multi_column_partitioning_1660101" found for row +DETAIL: Partition key of the failing row contains (c1, c2) = (20, -20). +CONTEXT: while executing command on localhost:57637 +-- see data is loaded to multi-column partitioned table +SELECT * FROM multi_column_partitioning ORDER BY 1, 2; + c1 | c2 +----+----- + 1 | 1 + 5 | -5 + 11 | -11 + 19 | -19 +(4 rows) + +-- +-- Tests for locks on partitioned tables +-- +CREATE TABLE partitioning_locks(id int, ref_id int, time date) PARTITION BY RANGE (time); +-- create its partitions +CREATE TABLE partitioning_locks_2009 PARTITION OF partitioning_locks FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +CREATE TABLE partitioning_locks_2010 PARTITION OF partitioning_locks FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +-- distribute partitioned table +SELECT create_distributed_table('partitioning_locks', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +-- test locks on router SELECT +BEGIN; +SELECT * FROM partitioning_locks WHERE id = 1 ORDER BY 1, 2; + id | ref_id | time +----+--------+------ +(0 rows) + +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+----------------- + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2010 | relation | AccessShareLock +(3 rows) + +COMMIT; +-- test locks on real-time SELECT +BEGIN; +SELECT * FROM partitioning_locks ORDER BY 1, 2; + id | ref_id | time +----+--------+------ +(0 rows) + +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+----------------- + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2010 | relation | AccessShareLock +(3 rows) + +COMMIT; +-- test locks on task-tracker SELECT +SET citus.task_executor_type TO 'task-tracker'; +BEGIN; +SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; + id | ref_id | time | id | ref_id | time +----+--------+------+----+--------+------ +(0 rows) + +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+----------------- + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2010 | relation | AccessShareLock +(3 rows) + +COMMIT; +RESET citus.task_executor_type; +-- test locks on INSERT +BEGIN; +INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01'); +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + +COMMIT; +-- test locks on UPDATE +BEGIN; +UPDATE partitioning_locks SET time = '2009-02-01' WHERE id = 1; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + +COMMIT; +-- test locks on DELETE +BEGIN; +DELETE FROM partitioning_locks WHERE id = 1; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + +COMMIT; +-- test locks on INSERT/SELECT +CREATE TABLE partitioning_locks_for_select(id int, ref_id int, time date); +SELECT create_distributed_table('partitioning_locks_for_select', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +BEGIN; +INSERT INTO partitioning_locks SELECT * FROM partitioning_locks_for_select; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock + partitioning_locks_for_select | relation | AccessShareLock +(7 rows) + +COMMIT; +-- test locks on coordinator INSERT/SELECT +BEGIN; +INSERT INTO partitioning_locks SELECT * FROM partitioning_locks_for_select LIMIT 5; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | RowExclusiveLock + partitioning_locks_for_select | relation | AccessShareLock +(5 rows) + +COMMIT; +-- test locks on multi-shard UPDATE +BEGIN; +UPDATE partitioning_locks SET time = '2009-03-01'; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+------------------ + partitioning_locks | relation | AccessShareLock + partitioning_locks | relation | RowExclusiveLock + partitioning_locks_2009 | relation | AccessShareLock + partitioning_locks_2009 | relation | RowExclusiveLock + partitioning_locks_2010 | relation | AccessShareLock + partitioning_locks_2010 | relation | RowExclusiveLock +(6 rows) + +COMMIT; +-- test locks on DDL +BEGIN; +ALTER TABLE partitioning_locks ADD COLUMN new_column int; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+--------------------- + partitioning_locks | relation | AccessExclusiveLock + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessExclusiveLock + partitioning_locks_2010 | relation | AccessExclusiveLock +(4 rows) + +COMMIT; +-- test locks on TRUNCATE +BEGIN; +TRUNCATE partitioning_locks; +SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; + relation | locktype | mode +-------------------------+----------+--------------------- + partitioning_locks | relation | AccessExclusiveLock + partitioning_locks | relation | AccessShareLock + partitioning_locks_2009 | relation | AccessExclusiveLock + partitioning_locks_2009 | relation | ShareLock + partitioning_locks_2010 | relation | AccessExclusiveLock + partitioning_locks_2010 | relation | ShareLock +(6 rows) + +COMMIT; +-- test shard resource locks with multi-shard UPDATE +BEGIN; +UPDATE partitioning_locks_2009 SET time = '2009-03-01'; +-- see the locks on parent table +SELECT + logicalrelid, + locktype, + mode +FROM + pg_locks AS l JOIN pg_dist_shard AS s +ON + l.objid = s.shardid +WHERE + logicalrelid IN ('partitioning_locks', 'partitioning_locks_2009', 'partitioning_locks_2010') AND + 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 + 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) + +COMMIT; +-- test shard resource locks with TRUNCATE +BEGIN; +TRUNCATE partitioning_locks_2009; +-- see the locks on parent table +SELECT + logicalrelid, + locktype, + mode +FROM + pg_locks AS l JOIN pg_dist_shard AS s +ON + l.objid = s.shardid +WHERE + logicalrelid IN ('partitioning_locks', 'partitioning_locks_2009', 'partitioning_locks_2010') AND + pid = pg_backend_pid() +ORDER BY + 1, 2, 3; + logicalrelid | locktype | mode +-------------------------+----------+----------- + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock + partitioning_locks_2009 | advisory | ShareLock +(4 rows) + +COMMIT; +-- test shard resource locks with INSERT/SELECT +BEGIN; +INSERT INTO partitioning_locks_2009 SELECT * FROM partitioning_locks WHERE time >= '2009-01-01' AND time < '2010-01-01'; +-- see the locks on parent table +SELECT + logicalrelid, + locktype, + mode +FROM + pg_locks AS l JOIN pg_dist_shard AS s +ON + l.objid = s.shardid +WHERE + logicalrelid IN ('partitioning_locks', 'partitioning_locks_2009', 'partitioning_locks_2010') AND + 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 + 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) + +COMMIT; +-- test partition-wise join +CREATE TABLE partitioning_hash_join_test(id int, subid int) PARTITION BY HASH(subid); +CREATE TABLE partitioning_hash_join_test_0 PARTITION OF partitioning_hash_join_test FOR VALUES WITH (MODULUS 3, REMAINDER 0); +CREATE TABLE partitioning_hash_join_test_1 PARTITION OF partitioning_hash_join_test FOR VALUES WITH (MODULUS 3, REMAINDER 1); +CREATE TABLE partitioning_hash_join_test_2 PARTITION OF partitioning_hash_join_test FOR VALUES WITH (MODULUS 3, REMAINDER 2); +SELECT create_distributed_table('partitioning_hash_join_test', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT success FROM run_command_on_workers('alter system set enable_mergejoin to off'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system set enable_nestloop to off'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system set enable_indexscan to off'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system set enable_indexonlyscan to off'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system set enable_partitionwise_join to off'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('select pg_reload_conf()'); + success +--------- + t + t +(2 rows) + +EXPLAIN (COSTS OFF) +SELECT * FROM partitioning_hash_test JOIN partitioning_hash_join_test USING (id, subid); + QUERY PLAN +-------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Gather + Workers Planned: 2 + -> Parallel Hash Join + Hash Cond: ((partitioning_hash_join_test.id = partitioning_hash_test_1.id) AND (partitioning_hash_join_test.subid = partitioning_hash_test_1.subid)) + -> Parallel Append + -> Parallel Seq Scan on partitioning_hash_join_test_0_1660133 partitioning_hash_join_test + -> Parallel Seq Scan on partitioning_hash_join_test_1_1660137 partitioning_hash_join_test_1 + -> Parallel Seq Scan on partitioning_hash_join_test_2_1660141 partitioning_hash_join_test_2 + -> Parallel Hash + -> Parallel Append + -> Parallel Seq Scan on partitioning_hash_test_1_1660020 partitioning_hash_test_1 + -> Parallel Seq Scan on partitioning_hash_test_0_1660016 partitioning_hash_test + -> Parallel Seq Scan on partitioning_hash_test_2_1660032 partitioning_hash_test_2 +(18 rows) + +-- set partition-wise join on +SELECT success FROM run_command_on_workers('alter system set enable_partitionwise_join to on'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('select pg_reload_conf()'); + success +--------- + t + t +(2 rows) + +SET enable_partitionwise_join TO on; +EXPLAIN (COSTS OFF) +SELECT * FROM partitioning_hash_test JOIN partitioning_hash_join_test USING (id, subid); + QUERY PLAN +------------------------------------------------------------------------------------------------------------------------------------------------------------------------------ + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Append + -> Hash Join + Hash Cond: ((partitioning_hash_join_test.id = partitioning_hash_test.id) AND (partitioning_hash_join_test.subid = partitioning_hash_test.subid)) + -> Seq Scan on partitioning_hash_join_test_0_1660133 partitioning_hash_join_test + -> Hash + -> Seq Scan on partitioning_hash_test_0_1660016 partitioning_hash_test + -> Hash Join + Hash Cond: ((partitioning_hash_test_1.id = partitioning_hash_join_test_1.id) AND (partitioning_hash_test_1.subid = partitioning_hash_join_test_1.subid)) + -> Seq Scan on partitioning_hash_test_1_1660020 partitioning_hash_test_1 + -> Hash + -> Seq Scan on partitioning_hash_join_test_1_1660137 partitioning_hash_join_test_1 + -> Hash Join + Hash Cond: ((partitioning_hash_join_test_2.id = partitioning_hash_test_2.id) AND (partitioning_hash_join_test_2.subid = partitioning_hash_test_2.subid)) + -> Seq Scan on partitioning_hash_join_test_2_1660141 partitioning_hash_join_test_2 + -> Hash + -> Seq Scan on partitioning_hash_test_2_1660032 partitioning_hash_test_2 +(21 rows) + +-- note that partition-wise joins only work when partition key is in the join +-- following join does not have that, therefore join will not be pushed down to +-- partitions +EXPLAIN (COSTS OFF) +SELECT * FROM partitioning_hash_test JOIN partitioning_hash_join_test USING (id); + QUERY PLAN +--------------------------------------------------------------------------------------------------------- + Custom Scan (Citus Adaptive) + Task Count: 4 + Tasks Shown: One of 4 + -> Task + Node: host=localhost port=57637 dbname=regression + -> Hash Join + Hash Cond: (partitioning_hash_join_test.id = partitioning_hash_test.id) + -> Append + -> Seq Scan on partitioning_hash_join_test_0_1660133 partitioning_hash_join_test + -> Seq Scan on partitioning_hash_join_test_1_1660137 partitioning_hash_join_test_1 + -> Seq Scan on partitioning_hash_join_test_2_1660141 partitioning_hash_join_test_2 + -> Hash + -> Append + -> Seq Scan on partitioning_hash_test_0_1660016 partitioning_hash_test + -> Seq Scan on partitioning_hash_test_1_1660020 partitioning_hash_test_1 + -> Seq Scan on partitioning_hash_test_2_1660032 partitioning_hash_test_2 +(16 rows) + +-- reset partition-wise join +SELECT success FROM run_command_on_workers('alter system reset enable_partitionwise_join'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system reset enable_mergejoin'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system reset enable_nestloop'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system reset enable_indexscan'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('alter system reset enable_indexonlyscan'); + success +--------- + t + t +(2 rows) + +SELECT success FROM run_command_on_workers('select pg_reload_conf()'); + success +--------- + t + t +(2 rows) + +RESET enable_partitionwise_join; +DROP TABLE +IF EXISTS + partitioning_test_2009, + partitioned_events_table, + partitioned_users_table, + list_partitioned_events_table, + multi_column_partitioning, + partitioning_locks, + partitioning_locks_for_select; +-- make sure we can create a partitioned table with streaming replication +SET citus.replication_model TO 'streaming'; +CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); +CREATE TABLE partitioning_test_2009 PARTITION OF partitioning_test FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +SELECT create_distributed_table('partitioning_test', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +DROP TABLE partitioning_test; +-- make sure we can attach partitions to a distributed table in a schema +CREATE SCHEMA partitioning_schema; +CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); +SELECT create_distributed_table('partitioning_schema."schema-test"', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE partitioning_schema."schema-test_2009"(id int, time date); +ALTER TABLE partitioning_schema."schema-test" ATTACH PARTITION partitioning_schema."schema-test_2009" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +-- attached partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +ORDER BY 1; + logicalrelid +---------------------------------------- + partitioning_schema."schema-test" + partitioning_schema."schema-test_2009" +(2 rows) + +SELECT + logicalrelid, count(*) +FROM + pg_dist_shard +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +----------------------------------------+------- + partitioning_schema."schema-test" | 4 + partitioning_schema."schema-test_2009" | 4 +(2 rows) + +DROP TABLE partitioning_schema."schema-test"; +-- make sure we can create partition of a distributed table in a schema +CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); +SELECT create_distributed_table('partitioning_schema."schema-test"', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE partitioning_schema."schema-test_2009" PARTITION OF partitioning_schema."schema-test" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +-- newly created partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +ORDER BY 1; + logicalrelid +---------------------------------------- + partitioning_schema."schema-test" + partitioning_schema."schema-test_2009" +(2 rows) + +SELECT + logicalrelid, count(*) +FROM + pg_dist_shard +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +----------------------------------------+------- + partitioning_schema."schema-test" | 4 + partitioning_schema."schema-test_2009" | 4 +(2 rows) + +DROP TABLE partitioning_schema."schema-test"; +-- make sure creating partitioned tables works while search_path is set +CREATE TABLE partitioning_schema."schema-test"(id int, time date) PARTITION BY RANGE (time); +SET search_path = partitioning_schema; +SELECT create_distributed_table('"schema-test"', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE partitioning_schema."schema-test_2009" PARTITION OF "schema-test" FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +-- newly created partition is distributed as well +SELECT + logicalrelid +FROM + pg_dist_partition +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +ORDER BY 1; + logicalrelid +-------------------- + "schema-test" + "schema-test_2009" +(2 rows) + +SELECT + logicalrelid, count(*) +FROM + pg_dist_shard +WHERE + logicalrelid IN ('partitioning_schema."schema-test"'::regclass, 'partitioning_schema."schema-test_2009"'::regclass) +GROUP BY + logicalrelid +ORDER BY + 1,2; + logicalrelid | count +--------------------+------- + "schema-test" | 4 + "schema-test_2009" | 4 +(2 rows) + +-- test we don't deadlock when attaching and detaching partitions from partitioned +-- tables with foreign keys +CREATE TABLE reference_table(id int PRIMARY KEY); +SELECT create_reference_table('reference_table'); + create_reference_table +------------------------ + +(1 row) + +CREATE TABLE reference_table_2(id int PRIMARY KEY); +SELECT create_reference_table('reference_table_2'); + create_reference_table +------------------------ + +(1 row) + +CREATE TABLE partitioning_test(id int, time date) PARTITION BY RANGE (time); +CREATE TABLE partitioning_test_2008 PARTITION OF partitioning_test FOR VALUES FROM ('2008-01-01') TO ('2009-01-01'); +CREATE TABLE partitioning_test_2009 (LIKE partitioning_test); +CREATE TABLE partitioning_test_2010 (LIKE partitioning_test); +CREATE TABLE partitioning_test_2011 (LIKE partitioning_test); +-- distributing partitioning_test will also distribute partitioning_test_2008 +SELECT create_distributed_table('partitioning_test', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('partitioning_test_2009', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('partitioning_test_2010', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +SELECT create_distributed_table('partitioning_test_2011', 'id'); + create_distributed_table +-------------------------- + +(1 row) + +ALTER TABLE partitioning_test ADD CONSTRAINT partitioning_reference_fkey + FOREIGN KEY (id) REFERENCES reference_table(id) ON DELETE CASCADE; +ALTER TABLE partitioning_test_2009 ADD CONSTRAINT partitioning_reference_fkey_2009 + FOREIGN KEY (id) REFERENCES reference_table(id) ON DELETE CASCADE; +INSERT INTO partitioning_test_2010 VALUES (1, '2010-02-01'); +-- This should fail because of foreign key constraint violation +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2010 + FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +ERROR: insert or update on table "partitioning_test_2010_1660191" violates foreign key constraint "partitioning_reference_fkey_1660179" +DETAIL: Key (id)=(1) is not present in table "reference_table_1660177". +CONTEXT: while executing command on localhost:57637 +-- Truncate, so attaching again won't fail +TRUNCATE partitioning_test_2010; +-- Attach a table which already has the same constraint +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2009 + FOR VALUES FROM ('2009-01-01') TO ('2010-01-01'); +-- Attach a table which doesn't have the constraint +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2010 + FOR VALUES FROM ('2010-01-01') TO ('2011-01-01'); +-- Attach a table which has a different constraint +ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2011 + FOR VALUES FROM ('2011-01-01') TO ('2012-01-01'); +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2008; +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2009; +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2010; +ALTER TABLE partitioning_test DETACH PARTITION partitioning_test_2011; +DROP TABLE partitioning_test, partitioning_test_2008, partitioning_test_2009, + partitioning_test_2010, partitioning_test_2011, + reference_table, reference_table_2; +DROP SCHEMA partitioning_schema CASCADE; +NOTICE: drop cascades to table "schema-test" +RESET SEARCH_PATH; +DROP TABLE IF EXISTS + partitioning_hash_test, + partitioning_hash_join_test, + partitioning_test_failure, + non_distributed_partitioned_table, + partitioning_test_foreign_key; diff --git a/src/test/regress/expected/multi_prepare_plsql.out b/src/test/regress/expected/multi_prepare_plsql.out index 4c01a183c..1bac37702 100644 --- a/src/test/regress/expected/multi_prepare_plsql.out +++ b/src/test/regress/expected/multi_prepare_plsql.out @@ -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 diff --git a/src/test/regress/expected/multi_prepare_sql.out b/src/test/regress/expected/multi_prepare_sql.out index b7e2a8e61..3ded28e5a 100644 --- a/src/test/regress/expected/multi_prepare_sql.out +++ b/src/test/regress/expected/multi_prepare_sql.out @@ -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; diff --git a/src/test/regress/expected/multi_query_directory_cleanup.out b/src/test/regress/expected/multi_query_directory_cleanup.out index 8547afd99..12d9c6cd3 100644 --- a/src/test/regress/expected/multi_query_directory_cleanup.out +++ b/src/test/regress/expected/multi_query_directory_cleanup.out @@ -215,55 +215,18 @@ FETCH 1 FROM c_19; (1 row) SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f; - f ------------------ - master_job_0007 - master_job_0008 - master_job_0009 - master_job_0010 - master_job_0011 - master_job_0012 - master_job_0013 - master_job_0014 - master_job_0015 - master_job_0016 - master_job_0017 - master_job_0018 - master_job_0019 - master_job_0020 - master_job_0021 - master_job_0022 - master_job_0023 - master_job_0024 - master_job_0025 - master_job_0026 -(20 rows) + f +--- +(0 rows) -- close first, 17th (first after re-alloc) and last cursor. CLOSE c_00; CLOSE c_16; CLOSE c_19; SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f; - f ------------------ - master_job_0008 - master_job_0009 - master_job_0010 - master_job_0011 - master_job_0012 - master_job_0013 - master_job_0014 - master_job_0015 - master_job_0016 - master_job_0017 - master_job_0018 - master_job_0019 - master_job_0020 - master_job_0021 - master_job_0022 - master_job_0024 - master_job_0025 -(17 rows) + f +--- +(0 rows) ROLLBACK; SELECT pg_ls_dir('base/pgsql_job_cache'); diff --git a/src/test/regress/expected/multi_query_directory_cleanup_0.out b/src/test/regress/expected/multi_query_directory_cleanup_0.out new file mode 100644 index 000000000..8547afd99 --- /dev/null +++ b/src/test/regress/expected/multi_query_directory_cleanup_0.out @@ -0,0 +1,273 @@ +-- +-- MULTI_QUERY_DIRECTORY_CLEANUP +-- +-- We execute sub-queries on worker nodes, and copy query results to a directory +-- on the master node for final processing. When the query completes or fails, +-- the resource owner should automatically clean up these intermediate query +-- result files. +SET citus.next_shard_id TO 810000; +SET citus.enable_unique_job_ids TO off; +BEGIN; +-- pg_ls_dir() displays jobids. We explicitly set the jobId sequence +-- here so that the regression output becomes independent of the +-- number of jobs executed prior to running this test. +SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT pg_ls_dir('base/pgsql_job_cache'); + pg_ls_dir +----------- +(0 rows) + +COMMIT; +SELECT pg_ls_dir('base/pgsql_job_cache'); + pg_ls_dir +----------- +(0 rows) + +BEGIN; +SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT pg_ls_dir('base/pgsql_job_cache'); + pg_ls_dir +----------- +(0 rows) + +ROLLBACK; +SELECT pg_ls_dir('base/pgsql_job_cache'); + pg_ls_dir +----------- +(0 rows) + +-- Test that multiple job directories are all cleaned up correctly, +-- both individually (by closing a cursor) and in bulk when ending a +-- transaction. +BEGIN; +DECLARE c_00 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_00; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_01 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_01; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_02 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_02; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_03 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_03; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_04 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_04; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_05 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_05; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_06 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_06; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_07 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_07; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_08 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_08; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_09 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_09; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_10 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_10; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_11 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_11; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_12 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_12; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_13 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_13; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_14 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_14; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_15 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_15; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_16 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_16; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_17 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_17; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_18 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_18; + revenue +--------------- + 22770844.7654 +(1 row) + +DECLARE c_19 CURSOR FOR SELECT sum(l_extendedprice * l_discount) as revenue FROM lineitem; +FETCH 1 FROM c_19; + revenue +--------------- + 22770844.7654 +(1 row) + +SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f; + f +----------------- + master_job_0007 + master_job_0008 + master_job_0009 + master_job_0010 + master_job_0011 + master_job_0012 + master_job_0013 + master_job_0014 + master_job_0015 + master_job_0016 + master_job_0017 + master_job_0018 + master_job_0019 + master_job_0020 + master_job_0021 + master_job_0022 + master_job_0023 + master_job_0024 + master_job_0025 + master_job_0026 +(20 rows) + +-- close first, 17th (first after re-alloc) and last cursor. +CLOSE c_00; +CLOSE c_16; +CLOSE c_19; +SELECT * FROM pg_ls_dir('base/pgsql_job_cache') f ORDER BY f; + f +----------------- + master_job_0008 + master_job_0009 + master_job_0010 + master_job_0011 + master_job_0012 + master_job_0013 + master_job_0014 + master_job_0015 + master_job_0016 + master_job_0017 + master_job_0018 + master_job_0019 + master_job_0020 + master_job_0021 + master_job_0022 + master_job_0024 + master_job_0025 +(17 rows) + +ROLLBACK; +SELECT pg_ls_dir('base/pgsql_job_cache'); + pg_ls_dir +----------- +(0 rows) + diff --git a/src/test/regress/expected/multi_real_time_transaction.out b/src/test/regress/expected/multi_real_time_transaction.out index 8486409bb..f4b965665 100644 --- a/src/test/regress/expected/multi_real_time_transaction.out +++ b/src/test/regress/expected/multi_real_time_transaction.out @@ -1,4 +1,7 @@ SET citus.next_shard_id TO 1610000; +-- enforce 1 connection per placement since +-- the tests are prepared for that +SET citus.force_max_query_parallelization TO ON; CREATE SCHEMA multi_real_time_transaction; SET search_path = 'multi_real_time_transaction'; SET citus.shard_replication_factor to 1; @@ -454,9 +457,11 @@ SELECT pg_reload_conf(); BEGIN; SET citus.select_opens_transaction_block TO off; -- This query would self-deadlock if it ran in a distributed transaction -SELECT id, pg_advisory_lock(15) FROM test_table ORDER BY id; - id | pg_advisory_lock -----+------------------ +-- we use a different advisory lock because previous tests +-- still holds the advisory locks since the sessions are still active +SELECT id, pg_advisory_xact_lock(16) FROM test_table ORDER BY id; + id | pg_advisory_xact_lock +----+----------------------- 1 | 2 | 3 | diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index bbec4e5b8..d28d41244 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -120,7 +120,7 @@ INSERT INTO articles_hash VALUES (47, 7, 'abeyance', 1772); INSERT INTO articles_hash VALUES (48, 8, 'alkylic', 18610); INSERT INTO articles_hash VALUES (49, 9, 'anyone', 2681); INSERT INTO articles_hash 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 VALUES (50, 10, 'anjanette', 19519); @@ -1060,7 +1060,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 @@ -1928,62 +1928,62 @@ SELECT master_create_distributed_table('articles_append', 'author_id', 'append') SET citus.shard_replication_factor TO 1; SELECT master_create_empty_shard('articles_append') AS shard_id \gset UPDATE pg_dist_shard SET shardmaxvalue = 100, shardminvalue=1 WHERE shardid = :shard_id; -SELECT author_id FROM articles_append - WHERE - substring('articles_append'::regclass::text, 1, 5) = 'hello' - ORDER BY - author_id - LIMIT 1; -DEBUG: Router planner does not support append-partitioned tables. -DEBUG: push down of limit count: 1 -WARNING: relation "public.articles_append" does not exist -CONTEXT: while executing command on localhost:57638 -WARNING: relation "public.articles_append" does not exist -CONTEXT: while executing command on localhost:57638 -WARNING: relation "public.articles_append" does not exist -CONTEXT: while executing command on localhost:57638 -ERROR: failed to execute task 1 +-- we execute the query within a function to consolidate the error messages +-- between different executors +CREATE FUNCTION raise_failed_execution_router(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'; + ELSIF SQLERRM LIKE '%could not receive query results%' THEN + RAISE 'Task failed to execute'; + END IF; +END; +$$LANGUAGE plpgsql; +SET client_min_messages TO ERROR; +\set VERBOSITY terse +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_append + WHERE + substring('articles_append'::regclass::text, 1, 5) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); +ERROR: Task failed to execute -- same query with where false but evaluation left to worker -SELECT author_id FROM articles_append - WHERE - substring('articles_append'::regclass::text, 1, 4) = 'hello' - ORDER BY - author_id - LIMIT 1; -DEBUG: Router planner does not support append-partitioned tables. -DEBUG: push down of limit count: 1 -WARNING: relation "public.articles_append" does not exist -CONTEXT: while executing command on localhost:57638 -WARNING: relation "public.articles_append" does not exist -CONTEXT: while executing command on localhost:57638 -WARNING: relation "public.articles_append" does not exist -CONTEXT: while executing command on localhost:57638 -ERROR: failed to execute task 1 +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_append + WHERE + substring('articles_append'::regclass::text, 1, 4) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); +ERROR: Task failed to execute -- same query on router planner with where false but evaluation left to worker -SELECT author_id FROM articles_single_shard_hash - WHERE - substring('articles_single_shard_hash'::regclass::text, 1, 4) = 'hello' - ORDER BY - author_id - LIMIT 1; -DEBUG: Creating router plan -DEBUG: Plan is router executable -WARNING: relation "public.articles_single_shard_hash" does not exist -CONTEXT: while executing command on localhost:57637 -ERROR: could not receive query results -SELECT author_id FROM articles_hash - WHERE - author_id = 1 - AND substring('articles_hash'::regclass::text, 1, 5) = 'hello' - ORDER BY - author_id - LIMIT 1; -DEBUG: Creating router plan -DEBUG: Plan is router executable -DETAIL: distribution column value: 1 -WARNING: relation "public.articles_hash" does not exist -CONTEXT: while executing command on localhost:57637 -ERROR: could not receive query results +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_single_shard_hash + WHERE + substring('articles_single_shard_hash'::regclass::text, 1, 4) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); +ERROR: Task failed to execute +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_hash + WHERE + author_id = 1 + AND substring('articles_hash'::regclass::text, 1, 5) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); +ERROR: Task failed to execute -- create a dummy function to be used in filtering CREATE OR REPLACE FUNCTION someDummyFunction(regclass) RETURNS text AS @@ -1999,8 +1999,6 @@ SELECT * FROM articles_hash ORDER BY author_id, id LIMIT 5; -DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: push down of limit count: 5 id | author_id | title | word_count ----+-----------+--------------+------------ 1 | 1 | arsenous | 9572 @@ -2011,18 +2009,16 @@ DEBUG: push down of limit count: 5 (5 rows) -- router plannable, errors -SELECT * FROM articles_hash - WHERE - someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 - ORDER BY - author_id, id - LIMIT 5; -DEBUG: Creating router plan -DEBUG: Plan is router executable -DETAIL: distribution column value: 1 -WARNING: relation "public.articles_hash" does not exist -CONTEXT: while executing command on localhost:57637 -ERROR: could not receive query results +SELECT raise_failed_execution_router($$ + SELECT * FROM articles_hash + WHERE + someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 + ORDER BY + author_id, id + LIMIT 5; +$$); +ERROR: Task failed to execute +\set VERBOSITY DEFAULT -- temporarily turn off debug messages before dropping the function SET client_min_messages TO 'NOTICE'; DROP FUNCTION someDummyFunction(regclass); diff --git a/src/test/regress/expected/multi_router_planner_fast_path.out b/src/test/regress/expected/multi_router_planner_fast_path.out index f93993f2a..993624f87 100644 --- a/src/test/regress/expected/multi_router_planner_fast_path.out +++ b/src/test/regress/expected/multi_router_planner_fast_path.out @@ -62,7 +62,7 @@ INSERT INTO articles_hash VALUES (1, 1, 'arsenous', 9572), (2, 2, 'abducing', (40, 10, 'attemper', 14976),(41, 1, 'aznavour', 11814),(42, 2, 'ausable', 15885),(43, 3, 'affixal', 12723), (44, 4, 'anteport', 16793),(45, 5, 'afrasia', 864),(46, 6, 'atlanta', 17702),(47, 7, 'abeyance', 1772), (48, 8, 'alkylic', 18610),(49, 9, 'anyone', 2681),(50, 10, 'anjanette', 19519); -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- test simple select for a single row SELECT * FROM articles_hash WHERE author_id = 10 AND id = 50; @@ -254,16 +254,15 @@ DETAIL: NOTICE from localhost:57637 (1 row) +-- do not print notices from workers since the order is not deterministic +SET client_min_messages TO DEFAULT; SELECT master_create_worker_shards('company_employees', 4, 1); -DEBUG: schema "fast_path_router_select" already exists, skipping -DETAIL: NOTICE from localhost:57638 -DEBUG: schema "fast_path_router_select" already exists, skipping -DETAIL: NOTICE from localhost:57637 master_create_worker_shards ----------------------------- (1 row) +SET client_min_messages TO 'DEBUG2'; INSERT INTO company_employees values(1, 1, 0); DEBUG: Creating router plan DEBUG: Plan is router executable @@ -1319,20 +1318,35 @@ BEGIN RETURN md5($1::text); END; $$ LANGUAGE 'plpgsql' IMMUTABLE; +-- we execute the query within a function to consolidate the error messages +-- between different executors +CREATE FUNCTION raise_failed_execution_f_router(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'; + ELSIF SQLERRM LIKE '%could not receive query results%' THEN + RAISE 'Task failed to execute'; + END IF; +END; +$$LANGUAGE plpgsql; +SET client_min_messages TO ERROR; +\set VERBOSITY terse -- fast path router plannable, but errors -SELECT * FROM articles_hash - WHERE - someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 - ORDER BY - author_id, id - LIMIT 5; -DEBUG: Distributed planning for a fast-path router query -DEBUG: Creating router plan -DEBUG: Plan is router executable -DETAIL: distribution column value: 1 -WARNING: relation "fast_path_router_select.articles_hash" does not exist -CONTEXT: while executing command on localhost:57637 -ERROR: could not receive query results +SELECT raise_failed_execution_f_router($$ + SELECT * FROM articles_hash + WHERE + someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 + ORDER BY + author_id, id + LIMIT 5; +$$); +ERROR: Task failed to execute +\set VERBOSITY DEFAULT +SET client_min_messages TO DEFAULT; -- temporarily turn off debug messages before dropping the function SET client_min_messages TO 'NOTICE'; DROP FUNCTION someDummyFunction(regclass); @@ -2221,8 +2235,9 @@ DROP TABLE articles_append; DROP TABLE collections_list; RESET search_path; DROP SCHEMA fast_path_router_select CASCADE; -NOTICE: drop cascades to 4 other objects +NOTICE: drop cascades to 5 other objects DETAIL: drop cascades to table fast_path_router_select.articles_hash +drop cascades to function fast_path_router_select.raise_failed_execution_f_router(text) drop cascades to function fast_path_router_select.author_articles_max_id(integer) drop cascades to function fast_path_router_select.author_articles_id_word_count(integer) drop cascades to view fast_path_router_select.test_view diff --git a/src/test/regress/expected/multi_select_distinct.out b/src/test/regress/expected/multi_select_distinct.out index 9360c6238..95f00a6c9 100644 --- a/src/test/regress/expected/multi_select_distinct.out +++ b/src/test/regress/expected/multi_select_distinct.out @@ -204,7 +204,7 @@ EXPLAIN (COSTS FALSE) -> HashAggregate Group Key: remote_scan.l_orderkey Filter: (COALESCE((pg_catalog.sum(remote_scan.worker_column_3))::bigint, '0'::bigint) > 5) - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -234,7 +234,7 @@ EXPLAIN (COSTS FALSE) Filter: (COALESCE((pg_catalog.sum(remote_scan.worker_column_3))::bigint, '0'::bigint) > 5) -> Sort Sort Key: remote_scan.l_orderkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -278,7 +278,7 @@ EXPLAIN (COSTS FALSE) Group Key: COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) -> HashAggregate Group Key: remote_scan.worker_column_2, remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -307,7 +307,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.worker_column_2, remote_scan.worker_column_3 -> Sort Sort Key: remote_scan.worker_column_2, remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -356,7 +356,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.l_suppkey, COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) -> HashAggregate Group Key: remote_scan.l_suppkey, remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -387,7 +387,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.l_suppkey, remote_scan.worker_column_3 -> Sort Sort Key: remote_scan.l_suppkey, remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -437,7 +437,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.l_suppkey, (pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)) -> HashAggregate Group Key: remote_scan.l_suppkey, remote_scan.worker_column_4 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -495,7 +495,7 @@ EXPLAIN (COSTS FALSE) Sort Key: remote_scan.worker_column_4, (pg_catalog.sum(((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))) / pg_catalog.sum(remote_scan.worker_column_3)) -> HashAggregate Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -524,7 +524,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4 -> Sort Sort Key: remote_scan.worker_column_3, remote_scan.worker_column_4 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -572,7 +572,7 @@ EXPLAIN (COSTS FALSE) Group Key: (sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision) -> HashAggregate Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -631,7 +631,7 @@ EXPLAIN (COSTS FALSE) Group Key: ((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)) -> HashAggregate Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -691,7 +691,7 @@ EXPLAIN (COSTS FALSE) Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey -> HashAggregate Group Key: remote_scan.l_orderkey, remote_scan.l_partkey, remote_scan.l_suppkey, remote_scan.l_linenumber, remote_scan.l_quantity, remote_scan.l_extendedprice, remote_scan.l_discount, remote_scan.l_tax, remote_scan.l_returnflag, remote_scan.l_linestatus, remote_scan.l_shipdate, remote_scan.l_commitdate, remote_scan.l_receiptdate, remote_scan.l_shipinstruct, remote_scan.l_shipmode, remote_scan.l_comment - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -723,7 +723,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.l_orderkey, remote_scan.l_partkey, remote_scan.l_suppkey, remote_scan.l_linenumber, remote_scan.l_quantity, remote_scan.l_extendedprice, remote_scan.l_discount, remote_scan.l_tax, remote_scan.l_returnflag, remote_scan.l_linestatus, remote_scan.l_shipdate, remote_scan.l_commitdate, remote_scan.l_receiptdate, remote_scan.l_shipinstruct, remote_scan.l_shipmode, remote_scan.l_comment -> Sort Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey, remote_scan.l_suppkey, remote_scan.l_linenumber, remote_scan.l_quantity, remote_scan.l_extendedprice, remote_scan.l_discount, remote_scan.l_tax, remote_scan.l_returnflag, remote_scan.l_linestatus, remote_scan.l_shipdate, remote_scan.l_commitdate, remote_scan.l_receiptdate, remote_scan.l_shipinstruct, remote_scan.l_shipmode, remote_scan.l_comment - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -789,7 +789,7 @@ EXPLAIN (COSTS FALSE) Group Key: COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint), COALESCE((pg_catalog.sum(remote_scan.count_1))::bigint, '0'::bigint) -> HashAggregate Group Key: remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -820,7 +820,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.worker_column_3 -> Sort Sort Key: remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -863,7 +863,7 @@ EXPLAIN (COSTS FALSE) Group Key: ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision) -> HashAggregate Group Key: remote_scan.worker_column_2 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -891,7 +891,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.worker_column_2 -> Sort Sort Key: remote_scan.worker_column_2 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -920,7 +920,7 @@ EXPLAIN (COSTS FALSE) Group Key: array_length(array_cat_agg(remote_scan.array_length), 1), array_cat_agg(remote_scan.array_agg) -> HashAggregate Group Key: remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -952,7 +952,7 @@ EXPLAIN (COSTS FALSE) Group Key: remote_scan.worker_column_3 -> Sort Sort Key: remote_scan.worker_column_3 - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1003,7 +1003,7 @@ EXPLAIN (COSTS FALSE) -> HashAggregate Group Key: remote_scan.l_partkey Filter: (COALESCE((pg_catalog.sum(remote_scan.worker_column_3))::bigint, '0'::bigint) > 2) - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1075,7 +1075,7 @@ EXPLAIN (COSTS FALSE) Sort Key: remote_scan.l_partkey, remote_scan.l_suppkey -> HashAggregate Group Key: remote_scan.l_partkey, remote_scan.l_suppkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1117,7 +1117,7 @@ EXPLAIN (COSTS FALSE) Unique -> Sort Sort Key: remote_scan.l_orderkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1172,7 +1172,7 @@ EXPLAIN (COSTS FALSE) -> Unique -> Sort Sort Key: remote_scan.l_partkey, remote_scan.l_orderkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1216,7 +1216,7 @@ $Q$); Unique -> Sort Sort Key: remote_scan.o_custkey, remote_scan.l_orderkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 (5 rows) @@ -1228,12 +1228,12 @@ EXPLAIN (COSTS FALSE) FROM lineitem_hash_part JOIN orders_hash_part ON (l_orderkey = o_orderkey) WHERE o_custkey < 15; $Q$); - coordinator_plan -------------------------------------------- + coordinator_plan +------------------------------------------ Unique -> Sort Sort Key: remote_scan.o_custkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 (5 rows) @@ -1294,7 +1294,7 @@ $Q$); Unique -> Sort Sort Key: remote_scan.o_custkey, remote_scan.l_orderkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 (5 rows) @@ -1372,7 +1372,7 @@ EXPLAIN (COSTS FALSE) Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey -> HashAggregate Group Key: remote_scan.l_orderkey, remote_scan.l_partkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1423,7 +1423,7 @@ EXPLAIN (COSTS FALSE) Sort Key: remote_scan.l_orderkey, remote_scan.cnt -> HashAggregate Group Key: remote_scan.l_orderkey, remote_scan.cnt - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1477,7 +1477,7 @@ EXPLAIN (COSTS FALSE) -> Unique -> Sort Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task @@ -1529,7 +1529,7 @@ EXPLAIN (COSTS FALSE) -> Unique -> Sort Sort Key: remote_scan.l_partkey, remote_scan.l_orderkey - -> Custom Scan (Citus Real-Time) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: One of 4 -> Task diff --git a/src/test/regress/expected/multi_sql_function.out b/src/test/regress/expected/multi_sql_function.out index 4b06b11aa..771e836ab 100644 --- a/src/test/regress/expected/multi_sql_function.out +++ b/src/test/regress/expected/multi_sql_function.out @@ -62,7 +62,7 @@ SELECT sql_test_no_4(); -- 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 plain SQL functions SELECT sql_test_no_1(); sql_test_no_1 diff --git a/src/test/regress/expected/multi_transaction_recovery.out b/src/test/regress/expected/multi_transaction_recovery.out index 8370caf46..65a316bd1 100644 --- a/src/test/regress/expected/multi_transaction_recovery.out +++ b/src/test/regress/expected/multi_transaction_recovery.out @@ -1,5 +1,8 @@ -- Tests for prepared transaction recovery SET citus.next_shard_id TO 1220000; +-- enforce 1 connection per placement since +-- the tests are prepared for that +SET citus.force_max_query_parallelization TO ON; -- Disable auto-recovery for the initial tests ALTER SYSTEM SET citus.recover_2pc_interval TO -1; SELECT pg_reload_conf(); @@ -27,6 +30,7 @@ BEGIN; CREATE TABLE should_be_sorted_into_middle (value int); PREPARE TRANSACTION 'citus_0_should_be_sorted_into_middle'; \c - - - :master_port +SET citus.force_max_query_parallelization TO ON; -- Add "fake" pg_dist_transaction records and run recovery INSERT INTO pg_dist_transaction VALUES (1, 'citus_0_should_commit'); INSERT INTO pg_dist_transaction VALUES (1, 'citus_0_should_be_forgotten'); @@ -57,10 +61,14 @@ SELECT count(*) FROM pg_tables WHERE tablename = 'should_commit'; (1 row) \c - - - :master_port +SET citus.force_max_query_parallelization TO ON; SET citus.shard_replication_factor TO 2; SET citus.shard_count TO 2; SET citus.multi_shard_commit_protocol TO '2pc'; --- create_distributed_table should add 2 recovery records (1 connection per node) +-- create_distributed_table may behave differently if shards +-- created via the executor or not, so not checking its value +-- may result multiple test outputs, so instead just make sure that +-- there are at least 2 entries CREATE TABLE test_recovery (x text); SELECT create_distributed_table('test_recovery', 'x'); create_distributed_table @@ -68,10 +76,10 @@ SELECT create_distributed_table('test_recovery', 'x'); (1 row) -SELECT count(*) FROM pg_dist_transaction; - count -------- - 2 +SELECT count(*) >= 2 FROM pg_dist_transaction; + ?column? +---------- + t (1 row) -- create_reference_table should add another 2 recovery records @@ -82,10 +90,10 @@ SELECT create_reference_table('test_recovery_ref'); (1 row) -SELECT count(*) FROM pg_dist_transaction; - count -------- - 4 +SELECT count(*) >= 4 FROM pg_dist_transaction; + ?column? +---------- + t (1 row) SELECT recover_prepared_transactions(); @@ -208,6 +216,87 @@ SELECT count(*) FROM pg_dist_transaction; 4 (1 row) +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +-- the same test with citus.force_max_query_parallelization=off +-- should be fine as well +SET citus.force_max_query_parallelization TO OFF; +BEGIN; +INSERT INTO test_recovery_single VALUES ('hello-0'); +INSERT INTO test_recovery_single VALUES ('hello-2'); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +-- slightly more complicated test with citus.force_max_query_parallelization=off +-- should be fine as well +SET citus.force_max_query_parallelization TO OFF; +BEGIN; +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-0'; + count +------- + 2 +(1 row) + +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-2'; + count +------- + 2 +(1 row) + +INSERT INTO test_recovery_single VALUES ('hello-0'); +INSERT INTO test_recovery_single VALUES ('hello-2'); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + +SELECT recover_prepared_transactions(); + recover_prepared_transactions +------------------------------- + 0 +(1 row) + +-- the same test as the above with citus.force_max_query_parallelization=on +-- should be fine as well +SET citus.force_max_query_parallelization TO ON; +BEGIN; +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-0'; + count +------- + 3 +(1 row) + +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-2'; + count +------- + 3 +(1 row) + +INSERT INTO test_recovery_single VALUES ('hello-0'); +INSERT INTO test_recovery_single VALUES ('hello-2'); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + count +------- + 2 +(1 row) + -- Test whether auto-recovery runs ALTER SYSTEM SET citus.recover_2pc_interval TO 10; SELECT pg_reload_conf(); diff --git a/src/test/regress/expected/multi_upsert.out b/src/test/regress/expected/multi_upsert.out index 0d54719ea..f05ecf242 100644 --- a/src/test/regress/expected/multi_upsert.out +++ b/src/test/regress/expected/multi_upsert.out @@ -25,7 +25,7 @@ INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT ON CONST INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT (part_key) DO UPDATE SET other_col = 2, third_col = 4; -- see the results -SELECT * FROM upsert_test; +SELECT * FROM upsert_test ORDER BY part_key ASC; part_key | other_col | third_col ----------+-----------+----------- 1 | 2 | 4 diff --git a/src/test/regress/expected/sequential_modifications.out b/src/test/regress/expected/sequential_modifications.out index 9a2dc7480..25de19f96 100644 --- a/src/test/regress/expected/sequential_modifications.out +++ b/src/test/regress/expected/sequential_modifications.out @@ -4,6 +4,10 @@ -- Note: this test should not be executed in parallel with -- other tests since we're relying on disabling 2PC recovery -- +-- We're also setting force_max_query_parallelization throughout +-- the tests because the test file has the assumption that +-- each placement is accessed with a seperate connection +SET citus.force_max_query_parallelization TO on; CREATE SCHEMA test_seq_ddl; SET search_path TO 'test_seq_ddl'; SET citus.next_shard_id TO 16000; diff --git a/src/test/regress/expected/subquery_and_cte.out b/src/test/regress/expected/subquery_and_cte.out index d594e67a8..d6316043c 100644 --- a/src/test/regress/expected/subquery_and_cte.out +++ b/src/test/regress/expected/subquery_and_cte.out @@ -553,7 +553,9 @@ FOR i IN 1..3 LOOP GROUP BY user_id, cte.value_2); EXCEPTION WHEN OTHERS THEN - IF SQLERRM LIKE 'failed to execute task%' THEN + IF SQLERRM LIKE 'more than one row returned by a subquery%%' THEN + errors_received := errors_received + 1; + ELSIF SQLERRM LIKE 'failed to execute task%' THEN errors_received := errors_received + 1; END IF; END; @@ -562,7 +564,7 @@ RAISE '(%/3) failed to execute one of the tasks', errors_received; END; $$; ERROR: (3/3) failed to execute one of the tasks -CONTEXT: PL/pgSQL function inline_code_block line 29 at RAISE +CONTEXT: PL/pgSQL function inline_code_block line 31 at RAISE SET client_min_messages TO DEFAULT; DROP SCHEMA subquery_and_ctes CASCADE; NOTICE: drop cascades to 5 other objects diff --git a/src/test/regress/expected/with_dml.out b/src/test/regress/expected/with_dml.out index ac05a4e03..45a482f6e 100644 --- a/src/test/regress/expected/with_dml.out +++ b/src/test/regress/expected/with_dml.out @@ -102,6 +102,10 @@ DEBUG: Plan 16 query after replacing subqueries and CTEs: SELECT tenant_id FROM -- not a very meaningful query -- but has two modifying CTEs along with another -- modify statement +-- We need to force 1 connection per placement +-- otherwise the coordinator insert select fails +-- since COPY cannot be executed +SET citus.force_max_query_parallelization TO on; WITH copy_to_other_table AS ( INSERT INTO distributed_table SELECT * @@ -131,6 +135,7 @@ DEBUG: generating subplan 20_1 for CTE copy_to_other_table: INSERT INTO with_dm DEBUG: generating subplan 20_2 for CTE main_table_deleted: DELETE FROM with_dml.distributed_table WHERE ((dept OPERATOR(pg_catalog.<) 10) AND (NOT (EXISTS (SELECT 1 FROM with_dml.second_distributed_table WHERE ((second_distributed_table.dept OPERATOR(pg_catalog.=) 1) AND (second_distributed_table.tenant_id OPERATOR(pg_catalog.=) distributed_table.tenant_id)))))) RETURNING tenant_id, dept DEBUG: generating subplan 20_3 for subquery SELECT main_table_deleted.tenant_id, main_table_deleted.dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('20_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) main_table_deleted EXCEPT SELECT copy_to_other_table.tenant_id, copy_to_other_table.dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('20_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) copy_to_other_table DEBUG: Plan 20 query after replacing subqueries and CTEs: SELECT tenant_id, dept FROM (SELECT intermediate_result.tenant_id, intermediate_result.dept FROM read_intermediate_result('20_3'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id text, dept integer)) citus_insert_select_subquery +SET citus.force_max_query_parallelization TO off; -- CTE inside the UPDATE statement UPDATE second_distributed_table diff --git a/src/test/regress/expected/with_modifying.out b/src/test/regress/expected/with_modifying.out index 4eb69e616..41306b7d7 100644 --- a/src/test/regress/expected/with_modifying.out +++ b/src/test/regress/expected/with_modifying.out @@ -188,14 +188,32 @@ cte_2 AS ( SELECT user_id, value_2 FROM users_table WHERE value_2 IN (3, 4, 5, 6) ) INSERT INTO modify_table (SELECT cte_1.user_id FROM cte_1 join cte_2 on cte_1.value_2=cte_2.value_2); +-- we execute the query within a function to consolidate the error messages +-- between different executors +CREATE FUNCTION raise_failed_execution_cte(query text) RETURNS void AS $$ +BEGIN + EXECUTE query; + EXCEPTION WHEN OTHERS THEN + IF SQLERRM LIKE '%more than one row returned by a subquery used as an expression%' 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; +SET client_min_messages TO ERROR; +\set VERBOSITY terse -- even if this is an INSERT...SELECT, the CTE is under SELECT -WITH cte AS ( - SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2) -) -INSERT INTO modify_table (SELECT (SELECT value_2 FROM cte GROUP BY value_2)); -WARNING: more than one row returned by a subquery used as an expression -CONTEXT: while executing command on localhost:57638 -ERROR: could not receive query results +-- function joins in CTE results can create lateral joins that are not supported +SELECT raise_failed_execution_cte($$ + WITH cte AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2) + ) + INSERT INTO modify_table (SELECT (SELECT value_2 FROM cte GROUP BY value_2)); +$$); +ERROR: Task failed to execute +SET client_min_messages TO DEFAULT; +\set VERBOSITY DEFAULT -- CTEs prior to any other modification should error out WITH cte AS ( SELECT value_2 FROM users_table WHERE user_id IN (1, 2, 3) @@ -702,8 +720,9 @@ ROLLBACK; (1 row) DROP SCHEMA with_modifying CASCADE; -NOTICE: drop cascades to 4 other objects +NOTICE: drop cascades to 5 other objects DETAIL: drop cascades to table users_table drop cascades to table anchor_table +drop cascades to function raise_failed_execution_cte(text) drop cascades to table modify_table drop cascades to table summary_table diff --git a/src/test/regress/expected/with_transactions.out b/src/test/regress/expected/with_transactions.out index f91d444b1..a1a601665 100644 --- a/src/test/regress/expected/with_transactions.out +++ b/src/test/regress/expected/with_transactions.out @@ -74,7 +74,7 @@ DEBUG: generating subplan 9_2 for CTE ids_inserted: INSERT INTO with_transactio DEBUG: Plan 9 query after replacing subqueries and CTEs: UPDATE with_transactions.raw_table SET created_at = 'Sat Feb 10 20:00:00 2001 PST'::timestamp with time zone WHERE ((tenant_id OPERATOR(pg_catalog.=) ANY (SELECT ids_inserted.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('9_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id integer)) ids_inserted)) AND (tenant_id OPERATOR(pg_catalog.<) (SELECT distinct_count.count FROM (SELECT intermediate_result.count FROM read_intermediate_result('9_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) distinct_count))) TRUNCATE second_raw_table; COMMIT; --- sequential insert followed by parallel update causes execution issues +-- sequential insert followed by parallel update works just fine WITH ids_inserted AS ( INSERT INTO raw_table VALUES (11, 1000, now()), (12, 1000, now()), (13, 1000, now()) RETURNING tenant_id diff --git a/src/test/regress/expected/with_transactions_9.out b/src/test/regress/expected/with_transactions_9.out new file mode 100644 index 000000000..b0df003d2 --- /dev/null +++ b/src/test/regress/expected/with_transactions_9.out @@ -0,0 +1,125 @@ +CREATE SCHEMA with_transactions; +SET search_path TO with_transactions, public; +SET citus.shard_count TO 4; +SET citus.next_placement_id TO 800000; +CREATE TABLE with_transactions.raw_table (tenant_id int, income float, created_at timestamptz); +SELECT create_distributed_table('raw_table', 'tenant_id'); + create_distributed_table +-------------------------- + +(1 row) + +CREATE TABLE with_transactions.second_raw_table (tenant_id int, income float, created_at timestamptz); +SELECT create_distributed_table('second_raw_table', 'tenant_id'); + create_distributed_table +-------------------------- + +(1 row) + +INSERT INTO + raw_table (tenant_id, income, created_at) +SELECT + i % 10, i * 10.0, timestamp '2014-01-10 20:00:00' + i * interval '1 day' +FROM + generate_series (0, 100) i; +INSERT INTO second_raw_table SELECT * FROM raw_table; +SET client_min_messages TO DEBUG1; +-- run a transaction which DELETE +BEGIN; + WITH ids_to_delete AS + ( + SELECT tenant_id FROM raw_table WHERE income < 250 + ), + deleted_ids AS + ( + DELETE FROM raw_table WHERE created_at < '2014-02-10 20:00:00' AND tenant_id IN (SELECT * from ids_to_delete) RETURNING tenant_id + ) + UPDATE raw_table SET income = income * 2 WHERE tenant_id IN (SELECT tenant_id FROM deleted_ids); +DEBUG: generating subplan 3_1 for CTE ids_to_delete: SELECT tenant_id FROM with_transactions.raw_table WHERE (income OPERATOR(pg_catalog.<) (250)::double precision) +DEBUG: generating subplan 3_2 for CTE deleted_ids: DELETE FROM with_transactions.raw_table WHERE ((created_at OPERATOR(pg_catalog.<) 'Mon Feb 10 20:00:00 2014 PST'::timestamp with time zone) AND (tenant_id OPERATOR(pg_catalog.=) ANY (SELECT ids_to_delete.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('3_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id integer)) ids_to_delete))) RETURNING tenant_id +DEBUG: Plan 3 query after replacing subqueries and CTEs: UPDATE with_transactions.raw_table SET income = (income OPERATOR(pg_catalog.*) (2)::double precision) WHERE (tenant_id OPERATOR(pg_catalog.=) ANY (SELECT deleted_ids.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('3_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id integer)) deleted_ids)) +ROLLBACK; +-- see that both UPDATE and DELETE commands are rollbacked +SELECT count(*) FROM raw_table; + count +------- + 101 +(1 row) + +SELECT max(income) FROM raw_table; + max +------ + 1000 +(1 row) + +-- multi-statement multi shard modifying statements should work +BEGIN; + SELECT count (*) FROM second_raw_table; + count +------- + 101 +(1 row) + + WITH distinct_count AS ( + SELECT count(DISTINCT created_at) FROM raw_table + ), + ids_inserted AS + ( + INSERT INTO raw_table VALUES (11, 1000, now()) RETURNING tenant_id + ) + UPDATE raw_table SET created_at = '2001-02-10 20:00:00' + WHERE tenant_id IN (SELECT tenant_id FROM ids_inserted) AND tenant_id < (SELECT count FROM distinct_count); +DEBUG: generating subplan 9_1 for CTE distinct_count: SELECT count(DISTINCT created_at) AS count FROM with_transactions.raw_table +DEBUG: generating subplan 9_2 for CTE ids_inserted: INSERT INTO with_transactions.raw_table (tenant_id, income, created_at) VALUES (11, 1000, now()) RETURNING tenant_id +DEBUG: Plan 9 query after replacing subqueries and CTEs: UPDATE with_transactions.raw_table SET created_at = 'Sat Feb 10 20:00:00 2001 PST'::timestamp with time zone WHERE ((tenant_id OPERATOR(pg_catalog.=) ANY (SELECT ids_inserted.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('9_2'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id integer)) ids_inserted)) AND (tenant_id OPERATOR(pg_catalog.<) (SELECT distinct_count.count FROM (SELECT intermediate_result.count FROM read_intermediate_result('9_1'::text, 'binary'::citus_copy_format) intermediate_result(count bigint)) distinct_count))) + TRUNCATE second_raw_table; +COMMIT; +-- sequential insert followed by parallel update works just fine +WITH ids_inserted AS +( + INSERT INTO raw_table VALUES (11, 1000, now()), (12, 1000, now()), (13, 1000, now()) RETURNING tenant_id +) +UPDATE raw_table SET created_at = '2001-02-10 20:00:00' WHERE tenant_id IN (SELECT tenant_id FROM ids_inserted); +DEBUG: generating subplan 12_1 for CTE ids_inserted: INSERT INTO with_transactions.raw_table (tenant_id, income, created_at) VALUES (11,1000,now()), (12,1000,now()), (13,1000,now()) RETURNING raw_table.tenant_id +DEBUG: Plan 12 query after replacing subqueries and CTEs: UPDATE with_transactions.raw_table SET created_at = 'Sat Feb 10 20:00:00 2001 PST'::timestamp with time zone WHERE (tenant_id OPERATOR(pg_catalog.=) ANY (SELECT ids_inserted.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('12_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id integer)) ids_inserted)) +-- make sure that everything committed +SELECT count(*) FROM raw_table; + count +------- + 105 +(1 row) + +SELECT count(*) FROM raw_table WHERE created_at = '2001-02-10 20:00:00'; + count +------- + 4 +(1 row) + +SELECT count(*) FROM second_raw_table; + count +------- + 0 +(1 row) + +-- sequential insert followed by a sequential real-time query should be fine +BEGIN; +SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; +WITH ids_inserted AS +( + INSERT INTO raw_table (tenant_id) VALUES (11), (12), (13), (14) RETURNING tenant_id +) +SELECT income FROM second_raw_table WHERE tenant_id IN (SELECT * FROM ids_inserted) ORDER BY 1 DESC LIMIT 3; +DEBUG: generating subplan 17_1 for CTE ids_inserted: INSERT INTO with_transactions.raw_table (tenant_id) VALUES (11), (12), (13), (14) RETURNING raw_table.tenant_id +DEBUG: Plan 17 query after replacing subqueries and CTEs: SELECT income FROM with_transactions.second_raw_table WHERE (tenant_id OPERATOR(pg_catalog.=) ANY (SELECT ids_inserted.tenant_id FROM (SELECT intermediate_result.tenant_id FROM read_intermediate_result('17_1'::text, 'binary'::citus_copy_format) intermediate_result(tenant_id integer)) ids_inserted)) ORDER BY income DESC LIMIT 3 +DEBUG: push down of limit count: 3 + income +-------- +(0 rows) + +ROLLBACK; +RESET client_min_messages; +RESET citus.shard_count; +DROP SCHEMA with_transactions CASCADE; +NOTICE: drop cascades to 2 other objects +DETAIL: drop cascades to table raw_table +drop cascades to table second_raw_table diff --git a/src/test/regress/failure_schedule b/src/test/regress/failure_schedule index 51af5fabb..076eeddc3 100644 --- a/src/test/regress/failure_schedule +++ b/src/test/regress/failure_schedule @@ -4,7 +4,6 @@ test: failure_test_helpers # this should only be run by pg_regress_multi, you don't need it test: failure_setup test: multi_test_helpers - test: failure_ddl test: failure_truncate test: failure_create_index_concurrently diff --git a/src/test/regress/isolation_schedule b/src/test/regress/isolation_schedule index 85870b9c8..44173d85b 100644 --- a/src/test/regress/isolation_schedule +++ b/src/test/regress/isolation_schedule @@ -9,14 +9,22 @@ test: isolation_master_update_node # isolation_cluster_management such that tests # that come later can be parallelized test: isolation_cluster_management + +# the following tests depend on the distributed +# transactionId, so should not be parallelized +# and no tests should be added before these +test: isolation_distributed_transaction_id +test: isolation_dump_global_wait_edges +test: isolation_citus_dist_activity + + test: isolation_dml_vs_repair isolation_copy_placement_vs_copy_placement test: isolation_concurrent_dml isolation_data_migration test: isolation_drop_shards isolation_copy_placement_vs_modification test: isolation_insert_vs_vacuum isolation_transaction_recovery -test: isolation_distributed_transaction_id isolation_progress_monitoring +test: isolation_progress_monitoring test: isolation_dump_local_wait_edges -test: isolation_dump_global_wait_edges test: isolation_replace_wait_function test: isolation_distributed_deadlock_detection @@ -43,7 +51,6 @@ test: isolation_delete_vs_all test: isolation_truncate_vs_all test: isolation_drop_vs_all test: isolation_ddl_vs_all -test: isolation_citus_dist_activity test: isolation_get_all_active_transactions test: isolation_validate_vs_insert test: isolation_insert_select_conflict diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 9291f7ac2..b200dfa17 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -66,7 +66,7 @@ test: multi_explain test: multi_basic_queries multi_complex_expressions multi_subquery multi_subquery_complex_queries multi_subquery_behavioral_analytics test: multi_subquery_complex_reference_clause multi_subquery_window_functions multi_view multi_sql_function multi_prepare_sql test: sql_procedure multi_function_in_join -test: multi_subquery_in_where_reference_clause full_join +test: multi_subquery_in_where_reference_clause full_join adaptive_executor test: multi_subquery_union multi_subquery_in_where_clause multi_subquery_misc test: multi_agg_distinct multi_agg_approximate_distinct multi_limit_clause_approximate multi_outer_join_reference multi_single_relation_subquery multi_prepare_plsql test: multi_reference_table multi_select_for_update relation_access_tracking diff --git a/src/test/regress/output/multi_alter_table_statements.source b/src/test/regress/output/multi_alter_table_statements.source index 4a3954028..a8aa8d7ec 100644 --- a/src/test/regress/output/multi_alter_table_statements.source +++ b/src/test/regress/output/multi_alter_table_statements.source @@ -668,7 +668,7 @@ COMMIT; WARNING: duplicate key value violates unique constraint "ddl_commands_command_key" DETAIL: Key (command)=(CREATE INDEX) already exists. CONTEXT: while executing command on localhost:57638 -WARNING: failed to commit critical transaction on localhost:57638, metadata is likely out of sync +WARNING: failed to commit transaction on localhost:57638 -- The block should have committed with a warning SELECT indexname, tablename FROM pg_indexes WHERE tablename = 'single_shard_items' ORDER BY 1; indexname | tablename diff --git a/src/test/regress/pg_regress_multi.pl b/src/test/regress/pg_regress_multi.pl index e153909b1..c4d9cd502 100755 --- a/src/test/regress/pg_regress_multi.pl +++ b/src/test/regress/pg_regress_multi.pl @@ -323,6 +323,9 @@ push(@pgOptions, '-c', "citus.remote_task_check_interval=1ms"); push(@pgOptions, '-c', "citus.shard_replication_factor=2"); push(@pgOptions, '-c', "citus.node_connection_timeout=${connectionTimeout}"); +# we disable slow start by default to encourage parallelism within tests +push(@pgOptions, '-c', "citus.executor_slow_start_interval=0ms"); + if ($useMitmproxy) { # make tests reproducible by never trying to negotiate ssl @@ -354,6 +357,7 @@ if ($followercluster) push(@pgOptions, '-c', "wal_level=replica"); } + # disable automatic distributed deadlock detection during the isolation testing # to make sure that we always get consistent test outputs. If we don't manually # (i.e., calling a UDF) detect the deadlocks, some sessions that do not participate diff --git a/src/test/regress/specs/isolation_citus_dist_activity.spec b/src/test/regress/specs/isolation_citus_dist_activity.spec index 79edf2584..b4eeea83c 100644 --- a/src/test/regress/specs/isolation_citus_dist_activity.spec +++ b/src/test/regress/specs/isolation_citus_dist_activity.spec @@ -8,8 +8,7 @@ setup SET citus.shard_replication_factor TO 1; SET citus.shard_count TO 4; - -- we don't want to see any entries related to 2PC recovery - SET citus.max_cached_conns_per_worker TO 0; + CREATE TABLE test_table(column1 int, column2 int); SELECT create_distributed_table('test_table', 'column1'); } @@ -24,9 +23,13 @@ session "s1" 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 "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-alter-table" @@ -59,9 +62,6 @@ session "s2" 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 "s2-rollback" @@ -71,12 +71,12 @@ step "s2-rollback" step "s2-sleep" { - SELECT pg_sleep(0.2); + SELECT pg_sleep(0.5); } 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; } @@ -85,9 +85,6 @@ session "s3" 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 "s3-rollback" @@ -97,12 +94,12 @@ step "s3-rollback" 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; } # we prefer to sleep before "s2-view-dist" so that we can ensure # the "wait_event" in the output doesn't change randomly (e.g., NULL to CliendRead etc.) -permutation "s1-begin" "s2-begin" "s3-begin" "s1-alter-table" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" -permutation "s1-begin" "s2-begin" "s3-begin" "s1-insert" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" -permutation "s1-begin" "s2-begin" "s3-begin" "s1-select" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" -permutation "s1-begin" "s2-begin" "s3-begin" "s1-select-router" "s2-sleep" "s2-view-dist" "s3-view-worker" "s2-rollback" "s1-commit" "s3-rollback" +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" +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" +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" +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" diff --git a/src/test/regress/specs/isolation_copy_placement_vs_modification.spec b/src/test/regress/specs/isolation_copy_placement_vs_modification.spec index ab1cbad69..2a206737f 100644 --- a/src/test/regress/specs/isolation_copy_placement_vs_modification.spec +++ b/src/test/regress/specs/isolation_copy_placement_vs_modification.spec @@ -21,6 +21,7 @@ session "s1" step "s1-begin" { BEGIN; + SET LOCAL citus.select_opens_transaction_block TO off; } # since test_copy_placement_vs_modification has rep > 1 simple select query doesn't hit all placements diff --git a/src/test/regress/specs/isolation_distributed_transaction_id.spec b/src/test/regress/specs/isolation_distributed_transaction_id.spec index 4379592a9..834bc512f 100644 --- a/src/test/regress/specs/isolation_distributed_transaction_id.spec +++ b/src/test/regress/specs/isolation_distributed_transaction_id.spec @@ -82,7 +82,7 @@ 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; ; diff --git a/src/test/regress/specs/isolation_multi_shard_modify_vs_all.spec b/src/test/regress/specs/isolation_multi_shard_modify_vs_all.spec index c4f5612c2..6e016f14f 100644 --- a/src/test/regress/specs/isolation_multi_shard_modify_vs_all.spec +++ b/src/test/regress/specs/isolation_multi_shard_modify_vs_all.spec @@ -96,7 +96,7 @@ step "s2-change_connection_mode_to_sequential" step "s2-select" { - SELECT * FROM users_test_table ORDER BY value_2; + SELECT * FROM users_test_table ORDER BY value_2, value_3; } step "s2-insert-to-table" diff --git a/src/test/regress/sql/adaptive_executor.sql b/src/test/regress/sql/adaptive_executor.sql new file mode 100644 index 000000000..3e61abd90 --- /dev/null +++ b/src/test/regress/sql/adaptive_executor.sql @@ -0,0 +1,37 @@ +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'); +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); +SELECT sum(result::bigint) FROM run_command_on_workers($$ + SELECT count(*) FROM pg_stat_activity + WHERE pid <> pg_backend_pid() AND query LIKE '%8010090%' +$$); +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); +SELECT sum(result::bigint) FROM run_command_on_workers($$ + SELECT count(*) FROM pg_stat_activity + WHERE pid <> pg_backend_pid() AND query LIKE '%8010090%' +$$); +END; + +DROP SCHEMA adaptive_executor CASCADE; diff --git a/src/test/regress/sql/failure_1pc_copy_append.sql b/src/test/regress/sql/failure_1pc_copy_append.sql index a9e74fec1..71d698567 100644 --- a/src/test/regress/sql/failure_1pc_copy_append.sql +++ b/src/test/regress/sql/failure_1pc_copy_append.sql @@ -1,5 +1,8 @@ SELECT citus.mitmproxy('conn.allow()'); +-- 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'; @@ -49,10 +52,12 @@ COPY copy_test FROM PROGRAM 'echo 0, 0 && echo 1, 1 && echo 2, 4 && echo 3, 9' W 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; + +SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()'); SELECT count(1) FROM copy_test; ---- 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() || ')'); 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 diff --git a/src/test/regress/sql/failure_1pc_copy_hash.sql b/src/test/regress/sql/failure_1pc_copy_hash.sql index a15415089..6f32ce6cc 100644 --- a/src/test/regress/sql/failure_1pc_copy_hash.sql +++ b/src/test/regress/sql/failure_1pc_copy_hash.sql @@ -1,5 +1,8 @@ SELECT citus.mitmproxy('conn.allow()'); +-- 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'; diff --git a/src/test/regress/sql/failure_connection_establishment.sql b/src/test/regress/sql/failure_connection_establishment.sql index dff93c430..74c3f2969 100644 --- a/src/test/regress/sql/failure_connection_establishment.sql +++ b/src/test/regress/sql/failure_connection_establishment.sql @@ -54,13 +54,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; SELECT name FROM r1 WHERE id = 2; @@ -68,6 +68,62 @@ SELECT name FROM r1 WHERE id = 2; -- connection to have been delayed and thus caused a timeout SELECT citus.dump_network_traffic(); +SELECT citus.mitmproxy('conn.allow()'); + +-- 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)'); +-- 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; +SELECT count(*) FROM products; + +-- use OFFSET 1 to prevent printing the line where source +-- is the worker +SELECT citus.dump_network_traffic() ORDER BY 1 OFFSET 1; + +SELECT citus.mitmproxy('conn.allow()'); +SET citus.shard_replication_factor TO 1; +CREATE TABLE single_replicatated(key int); +SELECT create_distributed_table('single_replicatated', 'key'); + +-- 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)'); +SELECT count(*) FROM single_replicatated; + +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()'); +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); +SELECT citus.mitmproxy('conn.delay(500)'); +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); + +-- show that INSERT went through +SELECT count(*) FROM products WHERE product_no = 100; + + RESET client_min_messages; -- verify get_global_active_transactions works when a timeout happens on a connection @@ -75,5 +131,6 @@ SELECT get_global_active_transactions(); SELECT citus.mitmproxy('conn.allow()'); +SET citus.node_connection_timeout TO DEFAULT; DROP SCHEMA fail_connect CASCADE; SET search_path TO default; diff --git a/src/test/regress/sql/failure_create_distributed_table_non_empty.sql b/src/test/regress/sql/failure_create_distributed_table_non_empty.sql index 46dbadcc8..7138ac15e 100644 --- a/src/test/regress/sql/failure_create_distributed_table_non_empty.sql +++ b/src/test/regress/sql/failure_create_distributed_table_non_empty.sql @@ -1,6 +1,11 @@ -- -- 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'; @@ -49,7 +54,8 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); -- 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() || ')'); SELECT create_distributed_table('test_table', 'id'); @@ -221,7 +227,8 @@ SELECT count(*) FROM pg_dist_shard WHERE logicalrelid='create_distributed_table_ SELECT run_command_on_workers($$SELECT count(*) FROM information_schema.schemata WHERE schema_name = 'create_distributed_table_non_empty_failure'$$); -- 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() || ')'); SELECT create_distributed_table('test_table', 'id'); diff --git a/src/test/regress/sql/failure_create_table.sql b/src/test/regress/sql/failure_create_table.sql index 35f4198eb..11e0dd5f6 100644 --- a/src/test/regress/sql/failure_create_table.sql +++ b/src/test/regress/sql/failure_create_table.sql @@ -39,7 +39,7 @@ SELECT count(*) FROM pg_dist_shard; 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$$); -- 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()'); SELECT create_distributed_table('test_table','id'); SELECT citus.mitmproxy('conn.allow()'); @@ -50,7 +50,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()'); SELECT create_distributed_table('test_table', 'id'); COMMIT; @@ -131,13 +131,14 @@ 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() || ')'); BEGIN; SELECT create_distributed_table('test_table','id'); COMMIT; - +SELECT recover_prepared_transactions(); SELECT citus.mitmproxy('conn.allow()'); SELECT count(*) FROM pg_dist_shard; 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$$); @@ -183,13 +184,13 @@ 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() || ')'); BEGIN; SELECT create_distributed_table('test_table','id'); COMMIT; - +SELECT recover_prepared_transactions(); SELECT citus.mitmproxy('conn.allow()'); SELECT count(*) FROM pg_dist_shard; 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$$); diff --git a/src/test/regress/sql/failure_ddl.sql b/src/test/regress/sql/failure_ddl.sql index ac6f547b5..4840fee22 100644 --- a/src/test/regress/sql/failure_ddl.sql +++ b/src/test/regress/sql/failure_ddl.sql @@ -9,6 +9,9 @@ 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; diff --git a/src/test/regress/sql/failure_multi_shard_update_delete.sql b/src/test/regress/sql/failure_multi_shard_update_delete.sql index 34c873ef2..60a4078e2 100644 --- a/src/test/regress/sql/failure_multi_shard_update_delete.sql +++ b/src/test/regress/sql/failure_multi_shard_update_delete.sql @@ -8,6 +8,9 @@ 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()'); CREATE TABLE t1(a int PRIMARY KEY, b int, c int); diff --git a/src/test/regress/sql/failure_real_time_select.sql b/src/test/regress/sql/failure_real_time_select.sql index 3dacca315..070e2d166 100644 --- a/src/test/regress/sql/failure_real_time_select.sql +++ b/src/test/regress/sql/failure_real_time_select.sql @@ -20,12 +20,12 @@ 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 public.raise_failed_execution('SELECT count(*) FROM test_table'); +SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").kill()'); +SELECT count(*) FROM test_table; 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()'); WITH results AS (SELECT * FROM test_table) SELECT * FROM test_table, results @@ -34,63 +34,64 @@ WHERE test_table.id = results.id; -- 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 public.raise_failed_execution('WITH - results AS (SELECT * FROM test_table) - SELECT * FROM test_table, results - WHERE test_table.id = results.id'); +SELECT citus.mitmproxy('conn.onQuery(query="SELECT|COPY").after(1).kill()'); +WITH results AS (SELECT * FROM test_table) +SELECT * FROM test_table, results +WHERE test_table.id = results.id; 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()'); SELECT count(*) FROM test_table; +-- 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() || ')'); SELECT count(*) FROM test_table; -- 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() || ')'); BEGIN; SELECT count(*) FROM test_table; 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() || ')'); BEGIN; UPDATE test_table SET value_1 = value_1 + 1; SELECT count(*) FROM test_table; 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() || ')'); WITH results AS (SELECT * FROM test_table) SELECT * FROM test_table WHERE test_table.id > (SELECT id FROM results); --- 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()'); BEGIN; SELECT count(*) FROM test_table; 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() || ')'); SELECT count(*) FROM test_table; -- 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() || ')'); BEGIN; SELECT count(*) FROM test_table; 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() || ')'); BEGIN; SELECT count(*) FROM test_table; COMMIT; @@ -101,6 +102,7 @@ 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()'); 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'); @@ -108,37 +110,37 @@ 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()'); SELECT count(*) FROM test_table; -- 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()'); BEGIN; SELECT count(*) FROM test_table; 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() || ')'); SELECT count(*) FROM test_table; -- 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() || ')'); BEGIN; SELECT count(*) FROM test_table; 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() || ')'); BEGIN; UPDATE test_table SET value_1 = value_1 + 1; SELECT count(*) FROM test_table; 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() || ')'); WITH results AS (SELECT * FROM test_table) SELECT * FROM test_table @@ -147,7 +149,7 @@ WHERE test_table.id > (SELECT id FROM results); -- 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()'); BEGIN; SELECT count(*) FROM test_table; COMMIT; diff --git a/src/test/regress/sql/failure_savepoints.sql b/src/test/regress/sql/failure_savepoints.sql index cf06048ad..a2b854215 100644 --- a/src/test/regress/sql/failure_savepoints.sql +++ b/src/test/regress/sql/failure_savepoints.sql @@ -1,3 +1,9 @@ +-- 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()'); SET citus.shard_count = 2; diff --git a/src/test/regress/sql/failure_truncate.sql b/src/test/regress/sql/failure_truncate.sql index 3d079d28b..a81ed753d 100644 --- a/src/test/regress/sql/failure_truncate.sql +++ b/src/test/regress/sql/failure_truncate.sql @@ -7,6 +7,12 @@ 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()'); -- we'll start with replication factor 1, 1PC and parallel mode diff --git a/src/test/regress/sql/failure_vacuum.sql b/src/test/regress/sql/failure_vacuum.sql index 79fb83dda..b51163ce4 100644 --- a/src/test/regress/sql/failure_vacuum.sql +++ b/src/test/regress/sql/failure_vacuum.sql @@ -1,3 +1,7 @@ +-- 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; @@ -23,6 +27,10 @@ SELECT citus.mitmproxy('conn.onQuery(query="^COMMIT").kill()'); 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); + UPDATE pg_dist_shard_placement SET shardstate = 1 WHERE shardid IN ( SELECT shardid FROM pg_dist_shard WHERE logicalrelid = 'vacuum_test'::regclass diff --git a/src/test/regress/sql/intermediate_results.sql b/src/test/regress/sql/intermediate_results.sql index 9279ebcc9..67e307f6f 100644 --- a/src/test/regress/sql/intermediate_results.sql +++ b/src/test/regress/sql/intermediate_results.sql @@ -38,16 +38,33 @@ JOIN (SELECT * FROM read_intermediate_result('squares', 'binary') AS res (x int, ORDER BY x; 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; +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; +$$); \set VERBOSITY DEFAULT +SET client_min_messages TO DEFAULT; -- try to read the file as text, will fail because of binary encoding BEGIN; diff --git a/src/test/regress/sql/limit_intermediate_size.sql b/src/test/regress/sql/limit_intermediate_size.sql index eef2f4301..64326847e 100644 --- a/src/test/regress/sql/limit_intermediate_size.sql +++ b/src/test/regress/sql/limit_intermediate_size.sql @@ -1,7 +1,7 @@ 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 ( @@ -39,24 +39,24 @@ LIMIT 10; -- 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; +(select count(*) as c from cte5) +) as foo; -- if we fetch the same amount of data at once, it should fail @@ -115,6 +115,7 @@ SELECT * FROM cte; -- 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) diff --git a/src/test/regress/sql/multi_binary_master_copy_format.sql b/src/test/regress/sql/multi_binary_master_copy_format.sql index 639de0941..785ed3d82 100644 --- a/src/test/regress/sql/multi_binary_master_copy_format.sql +++ b/src/test/regress/sql/multi_binary_master_copy_format.sql @@ -14,7 +14,7 @@ SET citus.task_executor_type TO 'task-tracker'; SELECT count(*) FROM lineitem; SELECT l_shipmode FROM lineitem WHERE l_partkey = 67310 OR l_partkey = 155190; -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; SELECT count(*) FROM lineitem; SELECT l_shipmode FROM lineitem WHERE l_partkey = 67310 OR l_partkey = 155190; diff --git a/src/test/regress/sql/multi_complex_expressions.sql b/src/test/regress/sql/multi_complex_expressions.sql index 678968caa..45dac7fe8 100644 --- a/src/test/regress/sql/multi_complex_expressions.sql +++ b/src/test/regress/sql/multi_complex_expressions.sql @@ -181,7 +181,7 @@ ORDER BY customer_keys.o_custkey DESC LIMIT 10 OFFSET 20; -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 diff --git a/src/test/regress/sql/multi_explain.sql b/src/test/regress/sql/multi_explain.sql index e526ec2de..29123d2a8 100644 --- a/src/test/regress/sql/multi_explain.sql +++ b/src/test/regress/sql/multi_explain.sql @@ -10,7 +10,7 @@ SELECT substring(:'server_version', '\d+')::int > 9 AS version_above_nine; \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 @@ -490,7 +490,7 @@ PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; -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; diff --git a/src/test/regress/sql/multi_extension.sql b/src/test/regress/sql/multi_extension.sql index 1015234d3..19a852472 100644 --- a/src/test/regress/sql/multi_extension.sql +++ b/src/test/regress/sql/multi_extension.sql @@ -362,7 +362,16 @@ 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'); +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; +$$; \c regression \c - - - :worker_1_port diff --git a/src/test/regress/sql/multi_follower_task_tracker.sql b/src/test/regress/sql/multi_follower_task_tracker.sql index 3a6fe215f..886be84d6 100644 --- a/src/test/regress/sql/multi_follower_task_tracker.sql +++ b/src/test/regress/sql/multi_follower_task_tracker.sql @@ -10,7 +10,7 @@ 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; SET citus.task_executor_type TO 'task-tracker'; SELECT * FROM tab; diff --git a/src/test/regress/sql/multi_function_in_join.sql b/src/test/regress/sql/multi_function_in_join.sql index fdd7549a8..964b66577 100644 --- a/src/test/regress/sql/multi_function_in_join.sql +++ b/src/test/regress/sql/multi_function_in_join.sql @@ -112,35 +112,58 @@ SELECT * FROM table1 JOIN max_and_min() m ON (m.maximum = data OR m.minimum = da -- 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; +$$); -- 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); +$$); + +SELECT raise_failed_execution_func_join($$ + SELECT * + FROM table1 + JOIN next_k_integers(10,5) WITH ORDINALITY next_integers + ON (id = next_integers.result); +$$); -- 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; +$$); RESET client_min_messages; DROP SCHEMA functions_in_joins CASCADE; SET search_path TO DEFAULT; diff --git a/src/test/regress/sql/multi_insert_select.sql b/src/test/regress/sql/multi_insert_select.sql index ef44fde08..ee028e2e1 100644 --- a/src/test/regress/sql/multi_insert_select.sql +++ b/src/test/regress/sql/multi_insert_select.sql @@ -1368,8 +1368,8 @@ 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; diff --git a/src/test/regress/sql/multi_modifications.sql b/src/test/regress/sql/multi_modifications.sql index fbb1d9d88..85e85a2e3 100644 --- a/src/test/regress/sql/multi_modifications.sql +++ b/src/test/regress/sql/multi_modifications.sql @@ -88,7 +88,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; SELECT * FROM append_partitioned WHERE id = 414123; SET client_min_messages TO DEFAULT; @@ -252,18 +252,26 @@ ALTER TABLE limit_orders_750000 RENAME TO renamed_orders; \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); --- 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; 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; diff --git a/src/test/regress/sql/multi_modifying_xacts.sql b/src/test/regress/sql/multi_modifying_xacts.sql index d8d3848fc..2d462e6bc 100644 --- a/src/test/regress/sql/multi_modifying_xacts.sql +++ b/src/test/regress/sql/multi_modifying_xacts.sql @@ -340,7 +340,7 @@ BEGIN; 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; SELECT lab_id FROM researchers WHERE lab_id = 2 AND id = 0; @@ -412,18 +412,18 @@ 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'); -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; SELECT * FROM labs WHERE id = 7; --- 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 @@ -435,13 +435,8 @@ AND s.logicalrelid = 'objects'::regclass; 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 @@ -464,7 +459,7 @@ BEGIN; INSERT INTO objects VALUES (1, 'apple'); INSERT INTO objects VALUES (2, 'BAD'); INSERT INTO labs VALUES (8, 'Aperture Science'); -INSERT INTO labs VALUES (9, 'BAD'); +INSERT INTO labs VALUES (2, 'BAD'); COMMIT; -- data should NOT be persisted @@ -501,7 +496,6 @@ COMMIT; -- data should be persisted SELECT * FROM objects WHERE id = 2; -SELECT * FROM labs WHERE id = 7; -- but one placement should be bad SELECT count(*) @@ -1074,12 +1068,12 @@ INSERT INTO users VALUES (3, 'burak'); \. 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; SELECT id FROM users WHERE id = 6; ALTER TABLE items ADD COLUMN last_update timestamptz; -END; +ROLLBACK; -- can perform sequential DDL after a co-located table has been read over 1 connection BEGIN; @@ -1089,7 +1083,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; @@ -1105,7 +1099,9 @@ BEGIN; -- now read from the reference table over each connection SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 2; SELECT user_id FROM items JOIN itemgroups ON (item_group = gid) WHERE user_id = 3; --- 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; END; diff --git a/src/test/regress/sql/multi_multiuser.sql b/src/test/regress/sql/multi_multiuser.sql index 0075f163a..d709dc61d 100644 --- a/src/test/regress/sql/multi_multiuser.sql +++ b/src/test/regress/sql/multi_multiuser.sql @@ -117,7 +117,7 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b. -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); -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); @@ -155,7 +155,7 @@ SELECT lock_relation_if_exists('test', 'ACCESS SHARE'); SELECT lock_relation_if_exists('test', 'EXCLUSIVE'); ABORT; -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; -- check no permission SET ROLE no_access; @@ -176,7 +176,7 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b. -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; -- should be able to use intermediate results as any user BEGIN; diff --git a/src/test/regress/sql/multi_mx_explain.sql b/src/test/regress/sql/multi_mx_explain.sql index 3a770befa..47eb4433f 100644 --- a/src/test/regress/sql/multi_mx_explain.sql +++ b/src/test/regress/sql/multi_mx_explain.sql @@ -9,7 +9,7 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1320000; \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; diff --git a/src/test/regress/sql/multi_mx_partitioning.sql b/src/test/regress/sql/multi_mx_partitioning.sql index 6030980a0..5dcc7fc44 100644 --- a/src/test/regress/sql/multi_mx_partitioning.sql +++ b/src/test/regress/sql/multi_mx_partitioning.sql @@ -53,7 +53,7 @@ ORDER BY 1,2; -- 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; \c - - - :master_port SET citus.replication_model TO 'streaming'; @@ -83,7 +83,7 @@ ORDER BY 1,2; -- 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; \c - - - :master_port SET citus.replication_model TO 'streaming'; @@ -122,7 +122,7 @@ ORDER BY 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; \c - - - :master_port SET citus.replication_model TO 'streaming'; @@ -144,7 +144,7 @@ ALTER TABLE partitioning_test ATTACH PARTITION partitioning_test_2013 FOR VALUES 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; \c - - - :master_port @@ -154,7 +154,7 @@ 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; -- make sure DROPping from worker node is not allowed DROP TABLE partitioning_test; diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index 8791d6234..d3de6de2e 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -62,7 +62,7 @@ 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 @@ -382,7 +382,7 @@ SELECT * WHERE author_id >= 1 AND author_id <= 3 ORDER BY 1,2,3,4; -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'; diff --git a/src/test/regress/sql/multi_null_minmax_value_pruning.sql b/src/test/regress/sql/multi_null_minmax_value_pruning.sql index cc317109f..3e308a9c7 100644 --- a/src/test/regress/sql/multi_null_minmax_value_pruning.sql +++ b/src/test/regress/sql/multi_null_minmax_value_pruning.sql @@ -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 diff --git a/src/test/regress/sql/multi_partitioning.sql b/src/test/regress/sql/multi_partitioning.sql index 4f707c4d1..fbe2ec280 100644 --- a/src/test/regress/sql/multi_partitioning.sql +++ b/src/test/regress/sql/multi_partitioning.sql @@ -872,7 +872,7 @@ BEGIN; SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; COMMIT; -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; -- test locks on INSERT BEGIN; diff --git a/src/test/regress/sql/multi_prepare_plsql.sql b/src/test/regress/sql/multi_prepare_plsql.sql index f61cdb06a..03ec9b7c4 100644 --- a/src/test/regress/sql/multi_prepare_plsql.sql +++ b/src/test/regress/sql/multi_prepare_plsql.sql @@ -176,7 +176,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(); @@ -420,7 +420,7 @@ SELECT task_tracker_partition_column_select(4); SELECT task_tracker_partition_column_select(5); SELECT task_tracker_partition_column_select(6); -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 $$ diff --git a/src/test/regress/sql/multi_prepare_sql.sql b/src/test/regress/sql/multi_prepare_sql.sql index 942a0efa2..db2eb8dd6 100644 --- a/src/test/regress/sql/multi_prepare_sql.sql +++ b/src/test/regress/sql/multi_prepare_sql.sql @@ -142,7 +142,7 @@ CREATE TEMP TABLE prepared_sql_test_7 AS EXECUTE prepared_test_7('UNITED KINGDOM SELECT * from prepared_sql_test_7; -- 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; @@ -402,7 +402,7 @@ EXECUTE prepared_task_tracker_partition_column_select(4); EXECUTE prepared_task_tracker_partition_column_select(5); EXECUTE prepared_task_tracker_partition_column_select(6); -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; -- check updates PREPARE prepared_partition_parameter_update(int, int) AS diff --git a/src/test/regress/sql/multi_real_time_transaction.sql b/src/test/regress/sql/multi_real_time_transaction.sql index 9c172f280..a94e0f8f8 100644 --- a/src/test/regress/sql/multi_real_time_transaction.sql +++ b/src/test/regress/sql/multi_real_time_transaction.sql @@ -1,5 +1,9 @@ SET citus.next_shard_id TO 1610000; +-- enforce 1 connection per placement since +-- the tests are prepared for that +SET citus.force_max_query_parallelization TO ON; + CREATE SCHEMA multi_real_time_transaction; SET search_path = 'multi_real_time_transaction'; SET citus.shard_replication_factor to 1; @@ -274,6 +278,7 @@ SET LOCAL citus.multi_shard_modify_mode TO 'sequential'; SELECT id, pg_advisory_lock(15) FROM test_table ORDER BY 1 DESC; ROLLBACK; + SET client_min_messages TO DEFAULT; alter system set deadlock_timeout TO DEFAULT; SELECT pg_reload_conf(); @@ -281,7 +286,9 @@ SELECT pg_reload_conf(); BEGIN; SET citus.select_opens_transaction_block TO off; -- This query would self-deadlock if it ran in a distributed transaction -SELECT id, pg_advisory_lock(15) FROM test_table ORDER BY id; +-- we use a different advisory lock because previous tests +-- still holds the advisory locks since the sessions are still active +SELECT id, pg_advisory_xact_lock(16) FROM test_table ORDER BY id; END; DROP SCHEMA multi_real_time_transaction CASCADE; diff --git a/src/test/regress/sql/multi_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index bafe85c44..dcc4a1f38 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -107,7 +107,7 @@ INSERT INTO articles_hash 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 @@ -476,7 +476,7 @@ SELECT * WHERE author_id >= 1 AND author_id <= 3 ORDER BY 1,2,3,4; -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'; @@ -842,36 +842,63 @@ SET citus.shard_replication_factor TO 1; SELECT master_create_empty_shard('articles_append') AS shard_id \gset UPDATE pg_dist_shard SET shardmaxvalue = 100, shardminvalue=1 WHERE shardid = :shard_id; -SELECT author_id FROM articles_append - WHERE - substring('articles_append'::regclass::text, 1, 5) = 'hello' - ORDER BY - author_id - LIMIT 1; +-- we execute the query within a function to consolidate the error messages +-- between different executors +CREATE FUNCTION raise_failed_execution_router(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'; + ELSIF SQLERRM LIKE '%could not receive query results%' THEN + RAISE 'Task failed to execute'; + END IF; +END; +$$LANGUAGE plpgsql; + +SET client_min_messages TO ERROR; +\set VERBOSITY terse + +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_append + WHERE + substring('articles_append'::regclass::text, 1, 5) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); -- same query with where false but evaluation left to worker -SELECT author_id FROM articles_append - WHERE - substring('articles_append'::regclass::text, 1, 4) = 'hello' - ORDER BY - author_id - LIMIT 1; +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_append + WHERE + substring('articles_append'::regclass::text, 1, 4) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); -- same query on router planner with where false but evaluation left to worker -SELECT author_id FROM articles_single_shard_hash - WHERE - substring('articles_single_shard_hash'::regclass::text, 1, 4) = 'hello' - ORDER BY - author_id - LIMIT 1; +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_single_shard_hash + WHERE + substring('articles_single_shard_hash'::regclass::text, 1, 4) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); -SELECT author_id FROM articles_hash - WHERE - author_id = 1 - AND substring('articles_hash'::regclass::text, 1, 5) = 'hello' - ORDER BY - author_id - LIMIT 1; +SELECT raise_failed_execution_router($$ + SELECT author_id FROM articles_hash + WHERE + author_id = 1 + AND substring('articles_hash'::regclass::text, 1, 5) = 'hello' + ORDER BY + author_id + LIMIT 1; +$$); -- create a dummy function to be used in filtering CREATE OR REPLACE FUNCTION someDummyFunction(regclass) @@ -891,12 +918,16 @@ SELECT * FROM articles_hash LIMIT 5; -- router plannable, errors -SELECT * FROM articles_hash - WHERE - someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 - ORDER BY - author_id, id - LIMIT 5; +SELECT raise_failed_execution_router($$ + SELECT * FROM articles_hash + WHERE + someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 + ORDER BY + author_id, id + LIMIT 5; +$$); + +\set VERBOSITY DEFAULT -- temporarily turn off debug messages before dropping the function SET client_min_messages TO 'NOTICE'; diff --git a/src/test/regress/sql/multi_router_planner_fast_path.sql b/src/test/regress/sql/multi_router_planner_fast_path.sql index 51bd57d68..b34b7cae0 100644 --- a/src/test/regress/sql/multi_router_planner_fast_path.sql +++ b/src/test/regress/sql/multi_router_planner_fast_path.sql @@ -67,7 +67,7 @@ INSERT INTO articles_hash VALUES (1, 1, 'arsenous', 9572), (2, 2, 'abducing', (44, 4, 'anteport', 16793),(45, 5, 'afrasia', 864),(46, 6, 'atlanta', 17702),(47, 7, 'abeyance', 1772), (48, 8, 'alkylic', 18610),(49, 9, 'anyone', 2681),(50, 10, 'anjanette', 19519); -SET citus.task_executor_type TO 'real-time'; +RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- test simple select for a single row @@ -123,7 +123,11 @@ SELECT * FROM id_author, id_title WHERE id_author.id = id_title.id; CREATE TABLE company_employees (company_id int, employee_id int, manager_id int); SELECT master_create_distributed_table('company_employees', 'company_id', 'hash'); + +-- do not print notices from workers since the order is not deterministic +SET client_min_messages TO DEFAULT; SELECT master_create_worker_shards('company_employees', 4, 1); +SET client_min_messages TO 'DEBUG2'; INSERT INTO company_employees values(1, 1, 0); INSERT INTO company_employees values(1, 2, 1); @@ -536,14 +540,37 @@ BEGIN END; $$ LANGUAGE 'plpgsql' IMMUTABLE; +-- we execute the query within a function to consolidate the error messages +-- between different executors +CREATE FUNCTION raise_failed_execution_f_router(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'; + ELSIF SQLERRM LIKE '%could not receive query results%' THEN + RAISE 'Task failed to execute'; + END IF; +END; +$$LANGUAGE plpgsql; + +SET client_min_messages TO ERROR; +\set VERBOSITY terse -- fast path router plannable, but errors -SELECT * FROM articles_hash - WHERE - someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 - ORDER BY - author_id, id - LIMIT 5; +SELECT raise_failed_execution_f_router($$ + SELECT * FROM articles_hash + WHERE + someDummyFunction('articles_hash') = md5('articles_hash') AND author_id = 1 + ORDER BY + author_id, id + LIMIT 5; +$$); + +\set VERBOSITY DEFAULT +SET client_min_messages TO DEFAULT; -- temporarily turn off debug messages before dropping the function SET client_min_messages TO 'NOTICE'; diff --git a/src/test/regress/sql/multi_sql_function.sql b/src/test/regress/sql/multi_sql_function.sql index d6dbc79a2..2ec661f8a 100644 --- a/src/test/regress/sql/multi_sql_function.sql +++ b/src/test/regress/sql/multi_sql_function.sql @@ -51,7 +51,7 @@ SELECT sql_test_no_4(); -- 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 plain SQL functions SELECT sql_test_no_1(); diff --git a/src/test/regress/sql/multi_transaction_recovery.sql b/src/test/regress/sql/multi_transaction_recovery.sql index 814a455a7..0c914d9f3 100644 --- a/src/test/regress/sql/multi_transaction_recovery.sql +++ b/src/test/regress/sql/multi_transaction_recovery.sql @@ -1,6 +1,10 @@ -- Tests for prepared transaction recovery SET citus.next_shard_id TO 1220000; +-- enforce 1 connection per placement since +-- the tests are prepared for that +SET citus.force_max_query_parallelization TO ON; + -- Disable auto-recovery for the initial tests ALTER SYSTEM SET citus.recover_2pc_interval TO -1; SELECT pg_reload_conf(); @@ -24,6 +28,7 @@ CREATE TABLE should_be_sorted_into_middle (value int); PREPARE TRANSACTION 'citus_0_should_be_sorted_into_middle'; \c - - - :master_port +SET citus.force_max_query_parallelization TO ON; -- Add "fake" pg_dist_transaction records and run recovery INSERT INTO pg_dist_transaction VALUES (1, 'citus_0_should_commit'); INSERT INTO pg_dist_transaction VALUES (1, 'citus_0_should_be_forgotten'); @@ -37,19 +42,23 @@ SELECT count(*) FROM pg_tables WHERE tablename = 'should_abort'; SELECT count(*) FROM pg_tables WHERE tablename = 'should_commit'; \c - - - :master_port +SET citus.force_max_query_parallelization TO ON; SET citus.shard_replication_factor TO 2; SET citus.shard_count TO 2; SET citus.multi_shard_commit_protocol TO '2pc'; --- create_distributed_table should add 2 recovery records (1 connection per node) +-- create_distributed_table may behave differently if shards +-- created via the executor or not, so not checking its value +-- may result multiple test outputs, so instead just make sure that +-- there are at least 2 entries CREATE TABLE test_recovery (x text); SELECT create_distributed_table('test_recovery', 'x'); -SELECT count(*) FROM pg_dist_transaction; +SELECT count(*) >= 2 FROM pg_dist_transaction; -- create_reference_table should add another 2 recovery records CREATE TABLE test_recovery_ref (x text); SELECT create_reference_table('test_recovery_ref'); -SELECT count(*) FROM pg_dist_transaction; +SELECT count(*) >= 4 FROM pg_dist_transaction; SELECT recover_prepared_transactions(); @@ -114,6 +123,45 @@ INSERT INTO test_recovery_single VALUES ('hello-2'); COMMIT; SELECT count(*) FROM pg_dist_transaction; +SELECT recover_prepared_transactions(); + +-- the same test with citus.force_max_query_parallelization=off +-- should be fine as well +SET citus.force_max_query_parallelization TO OFF; +BEGIN; +INSERT INTO test_recovery_single VALUES ('hello-0'); +INSERT INTO test_recovery_single VALUES ('hello-2'); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + +SELECT recover_prepared_transactions(); + +-- slightly more complicated test with citus.force_max_query_parallelization=off +-- should be fine as well +SET citus.force_max_query_parallelization TO OFF; +BEGIN; +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-0'; +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-2'; +INSERT INTO test_recovery_single VALUES ('hello-0'); +INSERT INTO test_recovery_single VALUES ('hello-2'); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + + +SELECT recover_prepared_transactions(); + +-- the same test as the above with citus.force_max_query_parallelization=on +-- should be fine as well +SET citus.force_max_query_parallelization TO ON; +BEGIN; +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-0'; +SELECT count(*) FROM test_recovery_single WHERE x = 'hello-2'; +INSERT INTO test_recovery_single VALUES ('hello-0'); +INSERT INTO test_recovery_single VALUES ('hello-2'); +COMMIT; +SELECT count(*) FROM pg_dist_transaction; + + -- Test whether auto-recovery runs ALTER SYSTEM SET citus.recover_2pc_interval TO 10; SELECT pg_reload_conf(); diff --git a/src/test/regress/sql/multi_upsert.sql b/src/test/regress/sql/multi_upsert.sql index 20af1e0cd..24503b7a4 100644 --- a/src/test/regress/sql/multi_upsert.sql +++ b/src/test/regress/sql/multi_upsert.sql @@ -31,7 +31,7 @@ INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1) ON CONFLICT (part_key) DO UPDATE SET other_col = 2, third_col = 4; -- see the results -SELECT * FROM upsert_test; +SELECT * FROM upsert_test ORDER BY part_key ASC; -- do a multi-row DO NOTHING insert INSERT INTO upsert_test (part_key, other_col) VALUES (1, 1), (2, 2) diff --git a/src/test/regress/sql/sequential_modifications.sql b/src/test/regress/sql/sequential_modifications.sql index 8c8c90a69..9d18ed904 100644 --- a/src/test/regress/sql/sequential_modifications.sql +++ b/src/test/regress/sql/sequential_modifications.sql @@ -4,6 +4,12 @@ -- Note: this test should not be executed in parallel with -- other tests since we're relying on disabling 2PC recovery -- +-- We're also setting force_max_query_parallelization throughout +-- the tests because the test file has the assumption that +-- each placement is accessed with a seperate connection +SET citus.force_max_query_parallelization TO on; + + CREATE SCHEMA test_seq_ddl; SET search_path TO 'test_seq_ddl'; SET citus.next_shard_id TO 16000; diff --git a/src/test/regress/sql/subquery_and_cte.sql b/src/test/regress/sql/subquery_and_cte.sql index f10892e61..cfdb03699 100644 --- a/src/test/regress/sql/subquery_and_cte.sql +++ b/src/test/regress/sql/subquery_and_cte.sql @@ -404,7 +404,9 @@ FOR i IN 1..3 LOOP GROUP BY user_id, cte.value_2); EXCEPTION WHEN OTHERS THEN - IF SQLERRM LIKE 'failed to execute task%' THEN + IF SQLERRM LIKE 'more than one row returned by a subquery%%' THEN + errors_received := errors_received + 1; + ELSIF SQLERRM LIKE 'failed to execute task%' THEN errors_received := errors_received + 1; END IF; END; diff --git a/src/test/regress/sql/with_dml.sql b/src/test/regress/sql/with_dml.sql index 4ba278f90..db1f42324 100644 --- a/src/test/regress/sql/with_dml.sql +++ b/src/test/regress/sql/with_dml.sql @@ -82,6 +82,11 @@ INSERT INTO distributed_table -- not a very meaningful query -- but has two modifying CTEs along with another -- modify statement + +-- We need to force 1 connection per placement +-- otherwise the coordinator insert select fails +-- since COPY cannot be executed +SET citus.force_max_query_parallelization TO on; WITH copy_to_other_table AS ( INSERT INTO distributed_table SELECT * @@ -106,6 +111,8 @@ INSERT INTO second_distributed_table SELECT * FROM copy_to_other_table; +SET citus.force_max_query_parallelization TO off; + -- CTE inside the UPDATE statement UPDATE second_distributed_table diff --git a/src/test/regress/sql/with_modifying.sql b/src/test/regress/sql/with_modifying.sql index 3d6479dcf..7261a2dec 100644 --- a/src/test/regress/sql/with_modifying.sql +++ b/src/test/regress/sql/with_modifying.sql @@ -109,12 +109,34 @@ cte_2 AS ( INSERT INTO modify_table (SELECT cte_1.user_id FROM cte_1 join cte_2 on cte_1.value_2=cte_2.value_2); --- even if this is an INSERT...SELECT, the CTE is under SELECT -WITH cte AS ( - SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2) -) -INSERT INTO modify_table (SELECT (SELECT value_2 FROM cte GROUP BY value_2)); +-- we execute the query within a function to consolidate the error messages +-- between different executors +CREATE FUNCTION raise_failed_execution_cte(query text) RETURNS void AS $$ +BEGIN + EXECUTE query; + EXCEPTION WHEN OTHERS THEN + IF SQLERRM LIKE '%more than one row returned by a subquery used as an expression%' 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; +SET client_min_messages TO ERROR; +\set VERBOSITY terse + +-- even if this is an INSERT...SELECT, the CTE is under SELECT +-- function joins in CTE results can create lateral joins that are not supported +SELECT raise_failed_execution_cte($$ + WITH cte AS ( + SELECT user_id, value_2 FROM users_table WHERE value_2 IN (1, 2) + ) + INSERT INTO modify_table (SELECT (SELECT value_2 FROM cte GROUP BY value_2)); +$$); + +SET client_min_messages TO DEFAULT; +\set VERBOSITY DEFAULT -- CTEs prior to any other modification should error out WITH cte AS ( diff --git a/src/test/regress/sql/with_transactions.sql b/src/test/regress/sql/with_transactions.sql index bb23ce195..78a288fc1 100644 --- a/src/test/regress/sql/with_transactions.sql +++ b/src/test/regress/sql/with_transactions.sql @@ -58,7 +58,7 @@ BEGIN; TRUNCATE second_raw_table; COMMIT; --- sequential insert followed by parallel update causes execution issues +-- sequential insert followed by parallel update works just fine WITH ids_inserted AS ( INSERT INTO raw_table VALUES (11, 1000, now()), (12, 1000, now()), (13, 1000, now()) RETURNING tenant_id