Commit Graph

69 Commits (4bc11c9bb4ecefc2f840dc5904b34d14500e4a35)

Author SHA1 Message Date
Marco Slot 232ad4735b Fix issues with insert..select casts and column ordering 2022-07-28 16:24:17 +02:00
Halil Ozan Akgul b01e7e884c Pass NULL for plannerInfo as we don't generate PlaceHolderVars 2021-09-03 15:27:25 +03:00
SaitTalhaNisanci 4559d02c41
Fix union pushdown issue (#5079)
* Fix UNION not being pushdown

Postgres optimizes column fields that are not needed in the output. We
were relying on these fields to understand if it is safe to push down a
union query.

This fix looks at the parse query, which has the original column fields
to detect if it is safe to push down a union query.

* Add more tests

* Simplify code and make it more robust

* Process varlevelsup > 0 in FindReferencedTableColumn

* Only look for outers vars in union path

* Add more comments

* Remove UNION ALL specific logic for pulling up childvars
2021-07-29 13:52:55 +03:00
SaitTalhaNisanci 03832f353c Drop postgres 11 support 2021-03-25 09:20:28 +03:00
Onur Tirtir cacb76d2c6
Not mention citus local tables in error messages (#4579) 2021-01-27 12:36:53 +03:00
Sait Talha Nisanci 5618f3a3fc Use BaseRestrictInfo for finding equality columns
Baseinfo also has pushed down filters etc, so it makes more sense to use
BaseRestrictInfo to determine what columns have constant equality
filters.

Also RteIdentity is used for removing conversion candidates instead of
rteIndex.
2020-12-15 18:18:36 +03:00
Hanefi Önaldı 6d8e83d24f
Replace worker_hash calls with partkey IS NOT NULL filters 2020-10-02 18:16:24 +03:00
Onur Tirtir 3a73fba810 Apply planner changes for citus local tables 2020-09-09 11:51:18 +03:00
SaitTalhaNisanci 366461ccdb
Introduce cache entry/table utilities (#4132)
Introduce table entry utility functions

Citus table cache entry utilities are introduced so that we can easily
extend existing functionality with minimum changes, specifically changes
to these functions. For example IsNonDistributedTableCacheEntry can be
extended for citus local tables without the need to scan the whole
codebase and update each relevant part.

* Introduce utility functions to find the type of tables

A table type can be a reference table, a hash/range/append distributed
table. Utility methods are created so that we don't have to worry about
how a table is considered as a reference table etc. This also makes it
easy to extend the table types.

* Add IsCitusTableType utilities

* Rename IsCacheEntryCitusTableType -> IsCitusTableTypeCacheEntry

* Change citus table types in some checks
2020-09-02 22:26:05 +03:00
SaitTalhaNisanci 73ef40886b
Rename FindNodeCheckXXX functions (#4106)
FindNodeCheck is not clear about what the function is doing. They are
renamed to FindNodeMatchingCheckFunctionXXX. Also for choosing elements in these
functions, CheckNodeFunc type is introduced.
2020-08-11 15:01:23 +03:00
Sait Talha Nisanci 8e9b52971c Use new var field names in the codebase
The codebase is updated to use varattnosync and varnosyn and we defined
the macros for older versions. This way we can just remove the macros
when we drop an older version.
2020-08-04 15:38:13 +03:00
Sait Talha Nisanci d68bfc5687 Improve error for index operator class parameters
The error message when index has opclassopts is improved and the commit
from postgres side is also included for future reference.

Also some minor style related changes are applied.
2020-08-04 15:38:13 +03:00
Sait Talha Nisanci 62879ee8c1 introduce planner_compat and pg_plan_query_compat macros
As the new planner and pg_plan_query_compat methods expect the query
string as well, macros are defined to be compatible in different
versions of postgres.

Relevant commit on Postgres:
6aba63ef3e606db71beb596210dd95fa73c44ce2

Command on Postgres:
git log --all --grep="pg_plan_query"
2020-08-04 15:10:22 +03:00
Sait Talha Nisanci bf831d2e59 Use table_openXXX methods in the codebase
With PG13 heap_* (heap_open, heap_close etc) are replaced with table_*
(table_open, table_close etc).

It is better to use the new table access methods in the codebase and
define the macros for the previous versions as we can easily remove the
macro without having to change the codebase when we drop the support for
the old version.

Commits that introduced this change on Postgres:
f25968c49697db673f6cd2a07b3f7626779f1827
e0c4ec07284db817e1f8d9adfb3fffc952252db0
4b21acf522d751ba5b6679df391d5121b6c4a35f

Command to see relevant commits on Postgres side:
git log --all --grep="heap_open"
2020-08-04 15:10:22 +03:00
Onder Kalaci c25de2cf22 Remove flag from
As it doesn't make any sense anymore
2020-07-20 12:45:05 +02:00
Jelte Fennema 759e628dd5
Handle some NULL issues that static analysis found (#4001)
Static analysis found some issues where we used the result from
ExtractResultRelationRTE, without checking that it wasn't NULL. It seems
like in all these cases it can never actually be NULL, since we have checked
before that it isn't a SELECT query. So, this PR is mostly to make static
analysis happy (and protect a bit against future changes of the code).
2020-07-09 15:46:42 +02:00
Marco Slot b4fec63bc0 Rename master evaluation to coordinator evaluation 2020-07-07 10:37:41 +02:00
Hadi Moshayedi 4ed59d2db3 Move more from insert_select_executor to insert_select_planner 2020-06-26 08:08:26 -07:00
Hadi Moshayedi 4e8d79998e Save INSERT/SELECT method in DistributedPlan.
This is so we don't need to calculate it twice in
insert_select_executor.c and multi_explain.c, which can
cause discrepancy if an update in one of them is not
reflected in the other site.
2020-06-25 08:55:48 -07:00
Marco Slot 2a3234ca26 Rename masterQuery to combineQuery 2020-06-17 14:14:37 +02:00
Halil Ozan Akgül 8c5eb6b7ea
Insert Select Into Local Table (#3870)
* Insert select with master query

* Use relid to set custom_scan_tlist varno

* Reviews

* Fixes null check

Co-authored-by: Marco Slot <marco.slot@gmail.com>
2020-06-12 17:06:31 +03:00
Philip Dubé c0515dcd67 This prepares for routing modifying CTEs, where modLevel should not be used to infer whether a plan is a select or not
SELECT_TASK is renamed to READ_TASK as a SELECT with modifying CTEs will be a MODIFYING_TASK

RouterInsertJob: Assert originalQuery->commandType == CMD_INSERT
CreateModifyPlan: Assert originalQuery->commandType != CMD_SELECT

Remove unused function IsModifyDistributedPlan

DistributedExecution, ExecutionParams, DistributedPlan: Rename hasReturning to expectResults
SELECTs set expectResults to true

Rename CreateSingleTaskRouterPlan to CreateSingleTaskRouterSelectPlan
2020-05-20 17:26:12 +00:00
Philip Dubé c00d57a955 CreateDistributedInsertSelectPlan: avoid calling GetCitusTableCacheEntry in a way that would invalidate live ShardInterval pointers 2020-04-17 14:44:23 +00:00
Marco Slot fd8cdb92f4 Evaluate nextval in the target list on the coordinator 2020-04-02 02:53:19 +02:00
SaitTalhaNisanci ba01f3457a
use macros for pg versions instead of hardcoded values (#3694)
3 Macros are defined for removing the hardcoded pg versions.
PG_VERSION_11, PG_VERSION_12 and PG_VERSION_13.
2020-04-01 17:01:52 +03:00
Philip Dubé 7cdfa1daab Rename LookupCitusTableCacheEntry to GetCitusTableCacheEntry, LookupLookupCitusTableCacheEntry back to LookupCitusTableCacheEntry 2020-03-08 14:08:23 +00:00
Philip Dubé a7cca1bcde Rename DistTableCacheEntry to CitusTableCacheEntry 2020-03-07 14:08:03 +00:00
Philip Dubé bec58000d6 Given IsDistributedTableRTE, there's ambiguity in what DistributedTable means
Elsewhere we used DistributedTable to include reference tables
Marco suggested we use CitusTable for distributed & reference tables

So renaming:
- IsDistributedTable -> IsCitusTable
- IsDistributedTableViaCatalog -> IsCitusTableViaCatalog
- DistributedTableCacheEntry -> CitusTableCacheEntry
- DistributedTableList -> CitusTableList
- isDistributedTable -> isCitusTable
- InsertSelectIntoDistributedTable -> InsertSelectIntoCitusTable
- ExtractFirstDistributedTableId -> ExtractFirstCitusTableId
2020-03-06 18:57:55 +00:00
Philip Dubé 3a906b8210 Fix typos noticed while reading through code trying to understand HAVING 2020-02-11 19:55:10 +00:00
Hadi Moshayedi 264530311a Don't use distributed insert/select for repartitioned joins 2020-02-03 13:13:30 -08:00
Philip Dubé 4b5d6c3ebe Rename RelayFileState to ShardState
Replace FILE_ prefix with SHARD_STATE_
2020-01-12 05:57:53 +00:00
Onder Kalaci ca293116fa Reduce calls to FastPathRouterQuery()
Before this commit, we called it twice durning planning. Instead,
we save the information and pass it.
2020-01-06 12:42:43 +01:00
Önder Kalacı a174eb4f7b
Do not go through standard_planner() for INSERTs (#3348)
That seems unnecessary. We already have the notion of FastPath queries,
simply add it there.
2020-01-03 12:15:22 +00:00
Marco Slot 133b8e1e0e Move coordinator insert..select logic into executor 2019-12-10 11:21:35 -08:00
SaitTalhaNisanci aeec3d1544
fix typo in dependent jobs and dependent task (#3244) 2019-11-28 23:47:28 +03:00
Jelte Fennema 1d8dde232f
Automatically convert useless declarations using regex replace (#3181)
* Add declaration removal to CI

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

* add function comment for IsLoggableLevel

* put ApplyLogRedaction to logutils
2019-11-15 14:59:13 +03:00
SaitTalhaNisanci 94a7e6475c
Remove copyright years (#2918)
* Update year as 2012-2019

* Remove copyright years
2019-10-15 17:44:30 +03:00
Philip Dubé 018ad1c58e pg12: version_compat.h, tuples, oids, misc 2019-08-22 18:57:23 +00:00
Philip Dubé 68c4b71f93 Fix up includes with pg12 changes 2019-08-22 18:56:21 +00:00
Philip Dubé b77c52f95b PlanRouterQuery: don't store list of list of shard intervals in relationShardList 2019-08-02 14:08:57 +00:00
Philip Dubé 0915027389 DistributedPlan: replace operation with modLevel
This causes no behaviorial changes, only organizes better to implement modifying CTEs

Also rename ExtactInsertRangeTableEntry to ExtractResultRelationRTE,
as the source of this function didn't match the documentation

Remove Task's upsertQuery in favor of ROW_MODIFY_NONCOMMUTATIVE

Split up AcquireExecutorShardLock into more internal functions

Tests: Normalize multi_reference_table multi_create_table_constraints
2019-07-16 13:58:18 -07:00
Önder Kalacı 40da78c6fd
Introduce the adaptive executor (#2798)
With this commit, we're introducing the Adaptive Executor. 


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

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



All the commits involved here:
* Initial unified executor prototype

* Latest changes

* Fix rebase conflicts to master branch

* Add missing variable for assertion

* Ensure that master_modify_multiple_shards() returns the affectedTupleCount

* Adjust intermediate result sizes

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

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

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

That might be useful under certain circumstances:

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

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

* For parameters to be resolved before using them

For the details, see PostgreSQL's copyParamList()

* Unified executor sorts the returning output

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

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

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

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

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

* Fix 2PC conditions for DDL tasks

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

* Support foreign keys to reference tables in unified executor

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

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

* Make sure to close the failed connections after the execution

* Improve comments

* Fix savepoints in unified executor.

* Rebuild the WaitEventSet only when necessary

* Unclaim connections on all errors.

* Improve failure handling for unified executor

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

Improve the logic to start a transaction block for distributed transactions

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

* Move waitEventSet to DistributedExecution

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

* Fix multi_insert_select test outputs

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

* Fix subeury_and_cte test

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

which is a lot clearer to the user.

* Fix intermediate_results test outputs

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

which makes a lot more sense.

* Fix multi_function_in_join test

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

* Fix multi_query_directory_cleanup test

The unified executor does not create any intermediate files.

* Fix with_transactions test

A test case that just started to work fine

* Fix multi_router_planner test outputs

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

which is a lot more clearer for the users

* Fix multi_router_planner_fast_path test

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

which is a lot more clearer for the users

* Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block

* Fix ordering in isolation_multi_shard_modify_vs_all

* Add executor locks to unified executor

* Make sure to allocate enought WaitEvents

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

* Fix rebase conflicts for master rebase

* Make sure that TRUNCATE relies on unified executor

* Implement true sequential execution for multi-row INSERTS

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

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

* Remove SESSION_LIFESPAN flag in unified_executor

* Apply failure test updates

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

* Unified executor honors citus.node_connection_timeout

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

And, as a side effect this fixes failure_connection_establishment
test.

* Properly increment/decrement pool size variables

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

* insert_select_executor goes through unified executor.

* Add missing file for task tracker

* Modify ExecuteTaskListExtended()'s signature

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

* Take partition locks correctly in unified executor

* Alternative implementation for force_max_query_parallelization

* Fix compile warnings in unified executor

* Fix style issues

* Decrement idleConnectionCount when idle connection is lost

* Always rebuild the wait event sets

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

* Make sure to allocate enough sized waitEventSet

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

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

* Implement SELECT FOR UPDATE on reference tables

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

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

* SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled

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

* Fix build error

* Fix the newConnectionCount calculation in ManageWorkerPool

* Fix rebase conflicts

* Fix minor test output differences

* Fix citus indent

* Remove duplicate sorts that is added with rebase

* Create distributed table via executor

* Fix wait flags in CheckConnectionReady

* failure_savepoints output for unified executor.

* failure_vacuum output (pg 10) for unified executor.

* Fix WaitEventSetWait timeout in unified executor

* Stabilize failure_truncate test output

* Add an ORDER BY to multi_upsert

* Fix regression test outputs after rebase to master

* Add executor.c comment

* Rename executor.c to adaptive_executor.c

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

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

* Implement a proper custom scan node for adaptive executor

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

* Add slow start interval to the executor

* Expose max_cached_connection_per_worker to user

* Do not start slow when there are cached connections

* Consider ExecutorSlowStartInterval in NextEventTimeout

* Fix memory issues with ReceiveResults().

* Disable executor via TaskExecutorType

* Make sure to execute the tests with the other executor

* Use task_executor_type to enable-disable adaptive executor

* Remove useless code

* Adjust the regression tests

* Add slow start regression test

* Rebase to master

* Fix test failures in adaptive executor.

* Rebase to master - 2

* Improve comments & debug messages

* Set force_max_query_parallelization in isolation_citus_dist_activity

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

* Adjust the default pool size

* Expand description of max_adaptive_executor_pool_size GUC

* Update warnings in FinishRemoteTransactionCommit()

* Improve session clean up at the end of execution

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

* Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all

* Add more ORDER BYs to multi_mx_partitioning
2019-06-28 14:04:40 +02:00
Philip Dubé 16886b3c63 Fix misc typos 2019-05-23 17:23:27 -07:00
Jason Petersen 71d5d1c865 Enable variable shadowing warnings; fix all
Rather than wait for another place like the previous commit to bite us,
I think we should turn on this warning.
2019-04-30 13:24:25 -06:00
Marco Slot 8893cc141d Support INSERT...SELECT with ON CONFLICT or RETURNING via coordinator
Before this commit, Citus supported INSERT...SELECT queries with
ON CONFLICT or RETURNING clauses only for pushdownable ones, since
queries supported via coordinator were utilizing COPY infrastructure
of PG to send selected tuples to the target worker nodes.

After this PR, INSERT...SELECT queries with ON CONFLICT or RETURNING
clauses will be performed in two phases via coordinator. In the first
phase selected tuples will be saved to the intermediate table which
is colocated with target table of the INSERT...SELECT query. Note that,
a utility function to save results to the colocated intermediate result
also implemented as a part of this commit. In the second phase, INSERT..
SELECT query is directly run on the worker node using the intermediate
table as the source table.
2018-11-30 15:29:12 +03:00
Jason Petersen 9fb951c312
Fix user-facing typos
Lintian found these (presumably by looking in the text section and
running them through e.g. aspell).
2018-10-09 16:54:03 -07:00
mehmet furkan şahin ef9f38b68d ApplyLogRedaction noop func is added 2018-08-17 14:48:54 -07:00
Nils Dijk 2d13900230
error on unsupported changing of distirbution column in ON CONFLICT for INSERT ... SELECT 2018-07-23 15:18:21 +02:00
Marco Slot fd4ff29f2f Add a debug message with distribution column value 2018-06-05 15:09:17 +03:00