mirror of https://github.com/citusdata/citus.git
10 Commits (a77ed9cd23864ac04c1993d19c05f50f3525cd69)
Author | SHA1 | Message | Date |
---|---|---|---|
|
a77ed9cd23
|
Refactor master query to be planned by postgres' planner (#3326)
DESCRIPTION: Replace the query planner for the coordinator part with the postgres planner Closes #2761 Citus had a simple rule based planner for the query executed on the query coordinator. This planner grew over time with the addigion of SQL support till it was getting close to the functionality of the postgres planner. Except the code was brittle and its complexity rose which made it hard to add new SQL support. Given its resemblance with the postgres planner it was a long outstanding wish to replace our hand crafted planner with the well supported postgres planner. This patch replaces our planner with a call to postgres' planner. Due to the functionality of the postgres planner we needed to support both projections and filters/quals on the citus custom scan node. When a sort operation is planned above the custom scan it might require fields to be reordered in the custom scan before returning the tuple (projection). The postgres planner assumes every custom scan node implements projections. Because we controlled the plan that was created we prevented reordering in the custom scan and never had implemented it before. A same optimisation applies to having clauses that could have been where clauses. Instead of applying the filter as a having on the aggregate it will push it down into the plan which could reach a custom scan node. For both filters and projections we have implemented them when tuples are read from the tuple store. If no projections or filters are required it will directly return the tuple from the tuple store. Otherwise it will loop tuples from the tuple store through the filter and projection until a tuple is found and returned. Besides filters being pushed down a side effect of having quals that could have been a where clause is that a call to read intermediate result could be called before the first tuple is fetched from the custom scan. This failed because the intermediate result would only be pulled to the coordinator on the first tuple fetch. To overcome this problem we do run the distributed subplans now before we run the postgres executor. This ensures the intermediate result is present on the coordinator in time. We do account for total time instrumentation by removing the instrumentation before handing control to the psotgres executor and update the timings our self. For future SQL support it is enough to create a valid query structure for the part of the query to be executed on the query coordinating node. As a utility we do serialise and print the query at debug level4 for engineers to inspect what kind of query is being planned on the query coordinator. |
|
|
0bf1e81e33 | Cache local plans on BeginScan | |
|
d82f3e9406
|
Introduce intermediate result broadcasting
In plain words, each distributed plan pulls the necessary intermediate results to the worker nodes that the plan hits. This is primarily useful in three ways. (i) If the distributed plan that uses intermediate result(s) is a router query, then the intermediate results are only broadcasted to a single node. (ii) If a distributed plan consists of only intermediate results, which is not uncommon, the intermediate results are broadcasted to a single node only. (iii) If a distributed query hits a sub-set of the shards in multiple workers, the intermediate results will be broadcasted to the relevant node(s). The final item (iii) becomes crucial for append/range distributed tables where typically the distributed queries hit a small subset of shards/workers. To do this, for each query that Citus creates a distributed plan, we keep track of the subPlans used in the queryTree, and save it in the distributed plan. Just before Citus executes each subPlan, Citus first keeps track of every worker node that the distributed plan hits, and marks every subPlan should be broadcasted to these nodes. Later, for each subPlan which is a distributed plan, Citus does this operation recursively since these distributed plans may access to different subPlans, and those have to be recorded as well. |
|
|
960cd02c67
|
Remove real time router executors (#3142)
* Remove unused executor codes All of the codes of real-time executor. Some functions in router executor still remains there because there are common functions. We'll move them to accurate places in the follow-up commits. * Move GUCs to transaction mngnt and remove unused struct * Update test output * Get rid of references of real-time executor from code * Warn if real-time executor is picked * Remove lots of unused connection codes * Removed unused code for connection restrictions Real-time and router executors cannot handle re-using of the existing connections within a transaction block. Adaptive executor and COPY can re-use the connections. So, there is no reason to keep the code around for applying the restrictions in the placement connection logic. |
|
|
94a7e6475c
|
Remove copyright years (#2918)
* Update year as 2012-2019 * Remove copyright years |
|
|
40da78c6fd
|
Introduce the adaptive executor (#2798)
With this commit, we're introducing the Adaptive Executor. The commit message consists of two distinct sections. The first part explains how the executor works. The second part consists of the commit messages of the individual smaller commits that resulted in this commit. The readers can search for the each of the smaller commit messages on https://github.com/citusdata/citus and can learn more about the history of the change. /*------------------------------------------------------------------------- * * adaptive_executor.c * * The adaptive executor executes a list of tasks (queries on shards) over * a connection pool per worker node. The results of the queries, if any, * are written to a tuple store. * * The concepts in the executor are modelled in a set of structs: * * - DistributedExecution: * Execution of a Task list over a set of WorkerPools. * - WorkerPool * Pool of WorkerSessions for the same worker which opportunistically * executes "unassigned" tasks from a queue. * - WorkerSession: * Connection to a worker that is used to execute "assigned" tasks * from a queue and may execute unasssigned tasks from the WorkerPool. * - ShardCommandExecution: * Execution of a Task across a list of placements. * - TaskPlacementExecution: * Execution of a Task on a specific placement. * Used in the WorkerPool and WorkerSession queues. * * Every connection pool (WorkerPool) and every connection (WorkerSession) * have a queue of tasks that are ready to execute (readyTaskQueue) and a * queue/set of pending tasks that may become ready later in the execution * (pendingTaskQueue). The tasks are wrapped in a ShardCommandExecution, * which keeps track of the state of execution and is referenced from a * TaskPlacementExecution, which is the data structure that is actually * added to the queues and describes the state of the execution of a task * on a particular worker node. * * When the task list is part of a bigger distributed transaction, the * shards that are accessed or modified by the task may have already been * accessed earlier in the transaction. We need to make sure we use the * same connection since it may hold relevant locks or have uncommitted * writes. In that case we "assign" the task to a connection by adding * it to the task queue of specific connection (in * AssignTasksToConnections). Otherwise we consider the task unassigned * and add it to the task queue of a worker pool, which means that it * can be executed over any connection in the pool. * * A task may be executed on multiple placements in case of a reference * table or a replicated distributed table. Depending on the type of * task, it may not be ready to be executed on a worker node immediately. * For instance, INSERTs on a reference table are executed serially across * placements to avoid deadlocks when concurrent INSERTs take conflicting * locks. At the beginning, only the "first" placement is ready to execute * and therefore added to the readyTaskQueue in the pool or connection. * The remaining placements are added to the pendingTaskQueue. Once * execution on the first placement is done the second placement moves * from pendingTaskQueue to readyTaskQueue. The same approach is used to * fail over read-only tasks to another placement. * * Once all the tasks are added to a queue, the main loop in * RunDistributedExecution repeatedly does the following: * * For each pool: * - ManageWorkPool evaluates whether to open additional connections * based on the number unassigned tasks that are ready to execute * and the targetPoolSize of the execution. * * Poll all connections: * - We use a WaitEventSet that contains all (non-failed) connections * and is rebuilt whenever the set of active connections or any of * their wait flags change. * * We almost always check for WL_SOCKET_READABLE because a session * can emit notices at any time during execution, but it will only * wake up WaitEventSetWait when there are actual bytes to read. * * We check for WL_SOCKET_WRITEABLE just after sending bytes in case * there is not enough space in the TCP buffer. Since a socket is * almost always writable we also use WL_SOCKET_WRITEABLE as a * mechanism to wake up WaitEventSetWait for non-I/O events, e.g. * when a task moves from pending to ready. * * For each connection that is ready: * - ConnectionStateMachine handles connection establishment and failure * as well as command execution via TransactionStateMachine. * * When a connection is ready to execute a new task, it first checks its * own readyTaskQueue and otherwise takes a task from the worker pool's * readyTaskQueue (on a first-come-first-serve basis). * * In cases where the tasks finish quickly (e.g. <1ms), a single * connection will often be sufficient to finish all tasks. It is * therefore not necessary that all connections are established * successfully or open a transaction (which may be blocked by an * intermediate pgbouncer in transaction pooling mode). It is therefore * essential that we take a task from the queue only after opening a * transaction block. * * When a command on a worker finishes or the connection is lost, we call * PlacementExecutionDone, which then updates the state of the task * based on whether we need to run it on other placements. When a * connection fails or all connections to a worker fail, we also call * PlacementExecutionDone for all queued tasks to try the next placement * and, if necessary, mark shard placements as inactive. If a task fails * to execute on all placements, the execution fails and the distributed * transaction rolls back. * * For multi-row INSERTs, tasks are executed sequentially by * SequentialRunDistributedExecution instead of in parallel, which allows * a high degree of concurrency without high risk of deadlocks. * Conversely, multi-row UPDATE/DELETE/DDL commands take aggressive locks * which forbids concurrency, but allows parallelism without high risk * of deadlocks. Note that this is unrelated to SEQUENTIAL_CONNECTION, * which indicates that we should use at most one connection per node, but * can run tasks in parallel across nodes. This is used when there are * writes to a reference table that has foreign keys from a distributed * table. * * Execution finishes when all tasks are done, the query errors out, or * the user cancels the query. * *------------------------------------------------------------------------- */ All the commits involved here: * Initial unified executor prototype * Latest changes * Fix rebase conflicts to master branch * Add missing variable for assertion * Ensure that master_modify_multiple_shards() returns the affectedTupleCount * Adjust intermediate result sizes The real-time executor uses COPY command to get the results from the worker nodes. Unified executor avoids that which results in less data transfer. Simply adjust the tests to lower sizes. * Force one connection per placement (or co-located placements) when requested The existing executors (real-time and router) always open 1 connection per placement when parallel execution is requested. That might be useful under certain circumstances: (a) User wants to utilize as much as CPUs on the workers per distributed query (b) User has a transaction block which involves COPY command Also, lots of regression tests rely on this execution semantics. So, we'd enable few of the tests with this change as well. * For parameters to be resolved before using them For the details, see PostgreSQL's copyParamList() * Unified executor sorts the returning output * Ensure that unified executor doesn't ignore sequential execution of DDLJob's Certain DDL commands, mainly creating foreign keys to reference tables, should be executed sequentially. Otherwise, we'd end up with a self distributed deadlock. To overcome this situaiton, we set a flag `DDLJob->executeSequentially` and execute it sequentially. Note that we have to do this because the command might not be called within a transaction block, and we cannot call `SetLocalMultiShardModifyModeToSequential()`. This fixes at least two test: multi_insert_select_on_conflit.sql and multi_foreign_key.sql Also, I wouldn't mind scattering local `targetPoolSize` variables within the code. The reason is that we'll soon have a GUC (or a global variable based on a GUC) that'd set the pool size. In that case, we'd simply replace `targetPoolSize` with the global variables. * Fix 2PC conditions for DDL tasks * Improve closing connections that are not fully established in unified execution * Support foreign keys to reference tables in unified executor The idea for supporting foreign keys to reference tables is simple: Keep track of the relation accesses within a transaction block. - If a parallel access happens on a distributed table which has a foreign key to a reference table, one cannot modify the reference table in the same transaction. Otherwise, we're very likely to end-up with a self-distributed deadlock. - If an access to a reference table happens, and then a parallel access to a distributed table (which has a fkey to the reference table) happens, we switch to sequential mode. Unified executor misses the function calls that marks the relation accesses during the execution. Thus, simply add the necessary calls and let the logic kick in. * Make sure to close the failed connections after the execution * Improve comments * Fix savepoints in unified executor. * Rebuild the WaitEventSet only when necessary * Unclaim connections on all errors. * Improve failure handling for unified executor - Implement the notion of errorOnAnyFailure. This is similar to Critical Connections that the connection managament APIs provide - If the nodes inside a modifying transaction expand, activate 2PC - Fix few bugs related to wait event sets - Mark placement INACTIVE during the execution as much as possible as opposed to we do in the COMMIT handler - Fix few bugs related to scheduling next placement executions - Improve decision on when to use 2PC Improve the logic to start a transaction block for distributed transactions - Make sure that only reference table modifications are always executed with distributed transactions - Make sure that stored procedures and functions are executed with distributed transactions * Move waitEventSet to DistributedExecution This could also be local to RunDistributedExecution(), but in that case we had to mark it as "volatile" to avoid PG_TRY()/PG_CATCH() issues, and cast it to non-volatile when doing WaitEventSetFree(). We thought that would make code a bit harder to read than making this non-local, so we move it here. See comments for PG_TRY() in postgres/src/include/elog.h and "man 3 siglongjmp" for more context. * Fix multi_insert_select test outputs Two things: 1) One complex transaction block is now supported. Simply update the test output 2) Due to dynamic nature of the unified executor, the orders of the errors coming from the shards might change (e.g., all of the queries on the shards would fail, but which one appears on the error message?). To fix that, we simply added it to our shardId normalization tool which happens just before diff. * Fix subeury_and_cte test The error message is updated from: failed to execute task To: more than one row returned by a subquery or an expression which is a lot clearer to the user. * Fix intermediate_results test outputs Simply update the error message from: could not receive query results to result "squares" does not exist which makes a lot more sense. * Fix multi_function_in_join test The error messages update from: Failed to execute task XXX To: function f(..) does not exist * Fix multi_query_directory_cleanup test The unified executor does not create any intermediate files. * Fix with_transactions test A test case that just started to work fine * Fix multi_router_planner test outputs The error message is update from: Could not receive query results To: Relation does not exists which is a lot more clearer for the users * Fix multi_router_planner_fast_path test The error message is update from: Could not receive query results To: Relation does not exists which is a lot more clearer for the users * Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block * Fix ordering in isolation_multi_shard_modify_vs_all * Add executor locks to unified executor * Make sure to allocate enought WaitEvents The previous code was missing the waitEvents for the latch and postmaster death. * Fix rebase conflicts for master rebase * Make sure that TRUNCATE relies on unified executor * Implement true sequential execution for multi-row INSERTS Execute the individual tasks executed one by one. Note that this is different than MultiShardConnectionType == SEQUENTIAL_CONNECTION case (e.g., sequential execution mode). In that case, running the tasks across the nodes in parallel is acceptable and implemented in that way. However, the executions that are qualified here would perform poorly if the tasks across the workers are executed in parallel. We currently qualify only one class of distributed queries here, multi-row INSERTs. If we do not enforce true sequential execution, concurrent multi-row upserts could easily form a distributed deadlock when the upserts touch the same rows. * Remove SESSION_LIFESPAN flag in unified_executor * Apply failure test updates We've changed the failure behaviour a bit, and also the error messages that show up to the user. This PR covers majority of the updates. * Unified executor honors citus.node_connection_timeout With this commit, unified executor errors out if even a single connection cannot be established within citus.node_connection_timeout. And, as a side effect this fixes failure_connection_establishment test. * Properly increment/decrement pool size variables Before this commit, the idle and active connection counts were not properly calculated. * insert_select_executor goes through unified executor. * Add missing file for task tracker * Modify ExecuteTaskListExtended()'s signature * Sort output of INSERT ... SELECT ... RETURNING * Take partition locks correctly in unified executor * Alternative implementation for force_max_query_parallelization * Fix compile warnings in unified executor * Fix style issues * Decrement idleConnectionCount when idle connection is lost * Always rebuild the wait event sets In the previous implementation, on waitFlag changes, we were only modifying the wait events. However, we've realized that it might be an over optimization since (a) we couldn't see any performance benefits (b) we see some errors on failures and because of (a) we prefer to disable it now. * Make sure to allocate enough sized waitEventSet With multi-row INSERTs, we might have more sessions than task*workerCount after few calls of RunDistributedExecution() because the previous sessions would also be alive. Instead, re-allocate events when the connectino set changes. * Implement SELECT FOR UPDATE on reference tables On master branch, we do two extra things on SELECT FOR UPDATE queries on reference tables: - Acquire executor locks - Execute the query on all replicas With this commit, we're implementing the same logic on the new executor. * SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled Otherwise, users would be very confused and their logic is very likely to break. * Fix build error * Fix the newConnectionCount calculation in ManageWorkerPool * Fix rebase conflicts * Fix minor test output differences * Fix citus indent * Remove duplicate sorts that is added with rebase * Create distributed table via executor * Fix wait flags in CheckConnectionReady * failure_savepoints output for unified executor. * failure_vacuum output (pg 10) for unified executor. * Fix WaitEventSetWait timeout in unified executor * Stabilize failure_truncate test output * Add an ORDER BY to multi_upsert * Fix regression test outputs after rebase to master * Add executor.c comment * Rename executor.c to adaptive_executor.c * Do not schedule tasks if the failed placement is not ready to execute Before the commit, we were blindly scheduling the next placement executions even if the failed placement is not on the ready queue. Now, we're ensuring that if failed placement execution is on a failed pool or session where the execution is on the pendingQueue, we do not schedule the next task. Because the other placement execution should be already running. * Implement a proper custom scan node for adaptive executor - Switch between the executors, add GUC to set the pool size - Add non-adaptive regression test suites - Enable CIRCLE CI for non-adaptive tests - Adjust test output files * Add slow start interval to the executor * Expose max_cached_connection_per_worker to user * Do not start slow when there are cached connections * Consider ExecutorSlowStartInterval in NextEventTimeout * Fix memory issues with ReceiveResults(). * Disable executor via TaskExecutorType * Make sure to execute the tests with the other executor * Use task_executor_type to enable-disable adaptive executor * Remove useless code * Adjust the regression tests * Add slow start regression test * Rebase to master * Fix test failures in adaptive executor. * Rebase to master - 2 * Improve comments & debug messages * Set force_max_query_parallelization in isolation_citus_dist_activity * Force max parallelization for creating shards when asked to use exclusive connection. * Adjust the default pool size * Expand description of max_adaptive_executor_pool_size GUC * Update warnings in FinishRemoteTransactionCommit() * Improve session clean up at the end of execution Explicitly list all the states that the execution might end, otherwise warn. * Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all * Add more ORDER BYs to multi_mx_partitioning |
|
|
85325e0098 | Refactor ScanStateGetExecutorState into its own function. | |
|
0b01c59fa6 | Refactor ScanStateGetTupleDescriptor() into a function. | |
|
16421f089f | Register citus custom scan nodes | |
|
83c1143505 |
Refactor custom scan related codes
In this commit, we don't change any codes, only create a new file and move the related functions and types there. |