Commit Graph

71 Commits (ce296ac62e1faf80243d8c2406328efd8c225f9a)

Author SHA1 Message Date
Philip Dubé 4e22f02997 Fix various typos due to zealous repetition 2021-03-04 19:28:15 +00:00
Onur Tirtir 253c19062a
Rename IsCitusInitiatedBackend to IsCitusInitiatedRemoteBackend (#4562) 2021-01-23 01:07:43 +03:00
Onur Tirtir 941c8fbf32
Automatically undistribute citus local tables when no more fkeys with reference tables (#4538) 2021-01-22 18:15:41 +03:00
Onur Tirtir 03bcccdee0
Fix hostname length check in StartNodeUserDatabaseConnection (#4363)
Copying string before hostname length check makes the check useless
2020-11-30 20:00:35 +03:00
Onur Tirtir 7f3d1182ed
Handle invalid connection hash entries (#4362)
If MemoryContextAlloc errors out -e.g. during an OOM-, ConnectionHashEntry->connections
stays as NULL.

With this commit, we add isValid flag to ConnectionHashEntry that should be set to true
right after we allocate & initialize ConnectionHashEntry->connections list properly, and we
check it before accesing to ConnectionHashEntry->connections.
2020-11-30 19:44:03 +03:00
Onur Tirtir f80f4839ad Remove unused functions that cppcheck found 2020-10-19 13:50:52 +03:00
Onder Kalaci eeb8c81de2 Implement shared connection count reservation & enable `citus.max_shared_pool_size` for COPY
With this patch, we introduce `locally_reserved_shared_connections.c/h` files
which are responsible for reserving some space in shared memory counters
upfront.

We sometimes need to reserve connections, but not necessarily
establish them. For example:
-  COPY command should reserve connections as it cannot know which
   connections it needs in which order. COPY establishes connections
   as any input data hits the workers. For example, for router COPY
   command, it only establishes 1 connection.

   As discussed here (https://github.com/citusdata/citus/pull/3849#pullrequestreview-431792473),
   COPY needs to reserve connections up-front, otherwise we can end
   up with resource starvation/un-detected deadlocks.
2020-08-03 18:51:40 +02:00
Onder Kalaci a2f53dff74 Make FindAvailableConnection() more strict
With adaptive connection management, we might have some connections
which are not fully initialized. Those connections should not be
qualified as available.
2020-07-23 15:59:50 +02:00
Jelte Fennema c6f5d5fe88
Add some asserts to pass static analysis (#3805) 2020-04-29 11:19:11 +02:00
Onder Kalaci bc54c5125f Increase the default value of citus.node_connection_timeout
The previous default was 5 seconds, and we change it to 30 seconds.
The main motivation for this is that for busy clusters, 5 seconds
can be too aggressive. Especially with connection throttling, the servers
might be kept busy for a really long time, and users may see the
connection errors more frequently.

We've done some sanity checks, for really quick queries (like
`SELECT count(*) from table`), 30 seconds is a decent value even
if users execute 300 distributed queries on the coordinator. We've
verified this on Hyperscale(Citus).
2020-04-24 15:16:42 +02:00
Marco Slot 8b83306a27 Issue worker messages with the same log level 2020-04-14 21:08:25 +02:00
Onder Kalaci aa6b641828 Throttle connections to the worker nodes
With this commit, we're introducing a new infrastructure to throttle
connections to the worker nodes. This infrastructure is useful for
multi-shard queries, router queries are have not been affected by this.

The goal is to prevent establishing more than citus.max_shared_pool_size
number of connections per worker node in total, across sessions.

To do that, we've introduced a new connection flag OPTIONAL_CONNECTION.
The idea is that some connections are optional such as the second
(and further connections) for the adaptive executor. A single connection
is enough to finish the distributed execution, the others are useful to
execute the query faster. Thus, they can be consider as optional connections.
When an optional connection is not allowed to the adaptive executor, it
simply skips it and continues the execution with the already established
connections. However, it'll keep retrying to establish optional
connections, in case some slots are open again.
2020-04-14 10:27:48 +02:00
Philip Dubé ab0b59ad3b GetConnParams: Set runtimeParamStart before setting keywords/values to avoid out of bounds access 2020-04-10 13:14:06 +00:00
Onder Kalaci 9b29a32d7a Remove all references for side channel connections
We don't need any side channel connections. That is actually
problematic in the sense that it creates extra connections.
Say, citus.max_adaptive_executor_pool_size equals to 1, Citus
ends up using one extra connection for the intermediate results.
Thus, not obeying citus.max_adaptive_executor_pool_size.

In this PR, we remove the following entities from the codebase
to allow further commits to implement not requiring extra connection
for the intermediate results:

- The connection flag REQUIRE_SIDECHANNEL
- The function GivePurposeToConnection
- The ConnectionPurpose struct and related fields
2020-04-07 17:06:55 +02:00
Jelte Fennema 2aabe3e2ef
Mark all connections for shutdown when citus.node_conninfo chan… (#3642)
We cache connections between nodes in our connection management code.
This is good for speed. For security this can be a problem though. If
the user changes settings related to TLS encryption they want those to
be applied to future queries. This is especially important when they did
not have TLS enabled before and now they want to enable it. This can
normally be achieved by changing citus.node_conninfo.  However, because
connections are not reopened there will still be old connections that
might not be encrypted at all.

This commit changes that by marking all connections to be shutdown at
the end of their current transaction. This way running transactions will
succeed, even if placement requires connections to be reused for this
transaction. But after this transaction completes any future statements
will use a connection created with the new connection options.

If a connection is requested and a connection is found that is marked
for shutdown, then we don't return this connection. Instead a new one is
created. This is needed to make sure that if there are no running
transactions, then the next statement will not use an old cached
connection, since connections are only actually shutdown at the end of a
transaction.
2020-03-24 15:31:41 +01:00
Onder Kalaci 7b4eb9611b Properly terminate connections at the end session
Citus coordinator (or MX nodes) caches `citus.max_cached_conns_per_worker` connections
per node. This means that, those connections are not terminated after each statement.
Instead, cached to avoid the cost of re-establishment. This is crucial for OLTP performance.

The problem with that approach is that, we never properly handle the termnation of
those cached connections. For instance, when a session on the coordinator disconnects,
you'd see the following logs on the workers:

```
2020-03-20 09:13:39.454 CET [64028] LOG:  could not receive data from client: Connection reset by peer
```

With this patch, we're terminating the cached connections properly at the end of the connection.
2020-03-20 17:34:34 +01:00
Philip Dubé 20abc4d2b5
Replace foreach with foreach_ptr/foreach_oid (#3544) 2020-02-27 16:54:49 +01:00
Jelte Fennema 8de8b62669 Convert unsafe APIs to safe ones 2020-02-25 15:39:27 +01:00
Philip Dubé 52042d4a00 Prefer instr_time to TimestampTz when we want CLOCK_MONOTONIC 2020-02-19 00:34:17 +00:00
Philip Dubé c252811884 dont: don't, wont: won't, acylic: acyclic 2020-02-05 17:32:22 +00:00
Philip Dubé fdcc413559 Code cleanup of adaptive_executor, connection_management, placement_connection
adaptive_executor: sort includes, use foreach_ptr, remove lies from FinishDistributedExecution docs
connection_management: rename msecs, which isn't milliseconds
placement_connection: small typos
2020-01-17 17:44:47 +00:00
Philip Dubé 73c06fae3b Introduce GetDistributeObjectOps to organize dispatch of logic dependent on node/object type 2020-01-09 18:24:29 +00:00
SaitTalhaNisanci 7ff4ce2169
Add adaptive executor support for repartition joins (#3169)
* WIP

* wip

* add basic logic to run a single job with repartioning joins with adaptive executor

* fix some warnings and return in ExecuteDependedTasks if there is none

* Add the logic to run depended jobs in adaptive executor

The execution of depended tasks logic is changed. With the current
logic:
- All tasks are created from the top level task list.
- At one iteration:
	- CurTasks whose dependencies are executed are found.
	- CurTasks are executed in parallel with adapter executor main
logic.
- The iteration is repeated until all tasks are completed.

* Separate adaptive executor repartioning logic

* Remove duplicate parts

* cleanup directories and schemas

* add basic repartion tests for adaptive executor

* Use the first placement to fetch data

In task tracker, when there are replicas, we try to fetch from a replica
for which a map task is succeeded. TaskExecution is used for this,
however TaskExecution is not used in adaptive executor. So we cannot use
the same thing as task tracker.

Since adaptive executor fails when a map task fails (There is no retry
logic yet). We know that if we try to execute a fetch task, all of its
map tasks already succeeded, so we can just use the first one to fetch
from.

* fix clean directories logic

* do not change the search path while creating a udf

* Enable repartition joins with adaptive executor with only enable_reparitition_joins guc

* Add comments to adaptive_executor_repartition

* dont run adaptive executor repartition test in paralle with other tests

* execute cleanup only in the top level execution

* do cleanup only in the top level ezecution

* not begin a transaction if repartition query is used

* use new connections for repartititon specific queries

New connections are opened to send repartition specific queries. The
opened connections will be closed at the FinishDistributedExecution.

While sending repartition queries no transaction is begun so that
we can see all changes.

* error if a modification was done prior to repartition execution

* not start a transaction if a repartition query and sql task, and clean temporary files and schemas at each subplan level

* fix cleanup logic

* update tests

* add missing function comments

* add test for transaction with DDL before repartition query

* do not close repartition connections in adaptive executor

* rollback instead of commit in repartition join test

* use close connection instead of shutdown connection

* remove unnecesary connection list, ensure schema owner before removing directory

* rename ExecuteTaskListRepartition

* put fetch query string in planner not executor as we currently support only replication factor = 1 with adaptive executor and repartition query and we know the query string in the planner phase in that case

* split adaptive executor repartition to DAG execution logic and repartition logic

* apply review items

* apply review items

* use an enum for remote transaction state and fix cleanup for repartition

* add outside transaction flag to find connections that are unclaimed instead of always opening a new transaction

* fix style

* wip

* rename removejobdir to partition cleanup

* do not close connections at the end of repartition queries

* do repartition cleanup in pg catch

* apply review items

* decide whether to use transaction or not at execution creation

* rename isOutsideTransaction and add missing comment

* not error in pg catch while doing cleanup

* use replication factor of the creation time, not current time to decide if task tracker should be chosen

* apply review items

* apply review items

* apply review item
2019-12-17 19:09:45 +03:00
Marco Slot 2f568ad5a5 Forbid using connections that sent intermediate results for data access and vice versa 2019-12-17 11:49:13 +01:00
SaitTalhaNisanci a0fe8646e0
add IsHoldOffCancellationReceived utility function (#3290) 2019-12-12 17:32:59 +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 306d159072
refactor AfterXacthodtConnectionHandling (#3202) 2019-11-19 14:50:23 +03: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
Jelte Fennema 1b2c438e69
Rename variables to not shadow globals in RHEL6 (#3194)
Fixes #2839
2019-11-15 12:12:24 +01:00
Önder Kalacı 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.
2019-11-05 12:48:10 +01:00
SaitTalhaNisanci 94a7e6475c
Remove copyright years (#2918)
* Update year as 2012-2019

* Remove copyright years
2019-10-15 17:44:30 +03:00
Marco Slot 35bef0f3db Avoid caching connections from backends that servicei internal connections 2019-09-28 08:32:10 +02:00
Hadi Moshayedi 76f3933b05 Add metadatasynced, and sync on master_update_node()
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
2019-09-18 09:32:54 -07:00
Marco Slot 4444d92dbc Set initial pool size to cached connection count 2019-07-23 20:40:32 +02: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
Nils Dijk eb98f2d13a
Fix null pointer caused by partial initialization of ConnParamsHashEntry (#2789)
It has been reported a null pointer dereference could be triggered in FreeConnParamsHashEntryFields. Likely cause is an error in GetConnParams which will leave the cached ConnParamsHashEntry in a state that would cause the null pointer dereference in a subsequent connection establishment to the same server. This has been simulated by inserting ereport(ERROR, ...) at certain places in the code.

Not only would ConnParamsHashEntry be in a state that would cause a crash, it was also leaking memory in the ConnectionContext due to the loss of pointers as they are only stored on the ConnParamsHashEntry at the end of the function.

This patch rewrites both the GetConnParams to store pointers 'durably' at every point in the code so that an error would not lose the pointer as well as FreeConnParamsHashEntryFields in a way that it can clear half initialised ConnParamsHashEntry's in a safer manner.
2019-06-21 18:16:43 +02:00
Hadi Moshayedi b240854b8c Use SendCancelationRequest() in ShutdownConnection() 2019-06-18 12:10:05 +02:00
Marco Slot bb3a96eacb Cache a configurable number of connections at xact end 2019-05-29 13:24:31 +02:00
Hadi Moshayedi b69a762e0b Fix savepoint rollback after multi-shard update failure. 2019-05-01 09:33:43 -07:00
Onder Kalaci 7d872a343a Rename MultiConnectionState to MultiConnectionPollState 2019-04-05 11:50:11 +03:00
Nils Dijk feaac69769
Implementation for asycn FinishConnectionListEstablishment (#2584) 2019-03-22 17:30:42 +01:00
Jason Petersen a2c6f596f9 Address code review comments 2019-03-21 11:59:52 -06:00
Jason Petersen 00d836e5a3 alloc non-global conn. params in provided context
Having DATA-segment string literals made blindly freeing the keywords/
values difficult, so I've switched to allocating all in the provided
context; because of this (and with the knowledge of the end point of
the global parameters), we can safely pfree non-global parameters when
we come across an invalid connection parameter entry.
2019-03-21 00:03:35 -06:00
Marco Slot 5ff1821411 Cache the current database name
Purely for performance reasons.
2019-03-20 12:14:46 +03:00
velioglu d1f005daac Adds UDFs for testing MX functionalities with isolation tests 2018-09-12 07:04:16 +03:00
mehmet furkan şahin ef9f38b68d ApplyLogRedaction noop func is added 2018-08-17 14:48:54 -07:00
Onder Kalaci d5472614df Use non-data connection for intermediate results
Make sure that intermediate results use a connection that is
not associated with any placement. That is useful in two ways:
    - More complex queries can be executed with CTEs
    - Safely use the same connections when there is a foreign key
      to reference table from a distributed table, which needs to
      use the same connection for modifications since the reference
      table might cascade to the distributed table.
2018-06-21 13:26:13 +03:00
Jason Petersen 7a75c2ed31 Add connparam invalidation trigger creation logic
This needs to live in Community, since we haven't yet added the com-
plication of having divergent upgrade scripts in Enterprise.
2018-06-20 14:13:18 -06:00
Jason Petersen 57b3f253c5
Add node_conninfo GUC and related logic
To support more flexible (i.e. not at compile-time) specification of
libpq connection parameters, this change adds a new GUC, node_conninfo,
which must be a space-separated string of key-value pairs suitable for
parsing by libpq's connection establishment methods.

To avoid rebuilding and parsing these values at connection time, this
change also adds a cache in front of the configuration params to permit
immediate use of any previously-calculated parameters.
2018-06-12 20:23:47 -06:00
mehmet furkan şahin d35f2725bf valgrind tests fix 2018-05-10 10:20:14 +03:00