Commit Graph

635 Commits (490765a7542aa4243b12981aa79bc3121220e009)

Author SHA1 Message Date
Önder Kalacı 8584cb005b
Do not evaluate functions on the coordinator for SELECT queries (#3440)
Previously, the logic for evaluting the functions and the parameters
were the same. That ended-up evaluting the functions inaccurately
on the coordinator. Instead, split the function evaluation logic
from parameter evalution logic.
2020-01-30 08:47:28 +01:00
SaitTalhaNisanci 94bd563ff0
switch back to old memory context in cache local plan for task (#3428) 2020-01-27 13:00:46 +03:00
Philip Dubé 50c5e814c8 CurrentDatabaseName: return const char* as we're borrowing from cache 2020-01-23 22:49:35 +00:00
Hadi Moshayedi 3e1004c232 Change DistributedResultFragment::nodeId to uint32.
This is to match the type of WorkerNode::nodeId.
2020-01-23 09:33:15 -08:00
Önder Kalacı ef7d1ea91d
Locally execute queries that don't need any data access (#3410)
* Update shardPlacement->nodeId to uint

As the source of the shardPlacement->nodeId is always workerNode->nodeId,
and that is uint32.

We had this hack because of: 0ea4e52df5 (r266421409)

And, that is gone with: 90056f7d3c (diff-c532177d74c72d3f0e7cd10e448ab3c6L1123)

So, we're safe to do it now.

* Relax the restrictions on using the local execution

Previously, whenever any local execution happens, we disabled further
commands to do any remote queries. The basic motivation for doing that
is to prevent any accesses in the same transaction block to access the
same placements over multiple sessions: one is local session the other
is remote session to the same placement.

However, the current implementation does not distinguish local accesses
being to a placement or not. For example, we could have local accesses
that only touches intermediate results. In that case, we should not
implement the same restrictions as they become useless.

So, this is a pre-requisite for executing the intermediate result only
queries locally.

* Update the error messages

As the underlying implementation has changed, reflect it in the error
messages.

* Keep track of connections to local node

With this commit, we're adding infrastructure to track if any connection
to the same local host is done or not.

The main motivation for doing this is that we've previously were more
conservative about not choosing local execution. Simply, we disallowed
local execution if any connection to any remote node is done. However,
if we want to use local execution for intermediate result only queries,
this'd be annoying because we expect all queries to touch remote node
before the final query.

Note that this approach is still limiting in Citus MX case, but for now
we can ignore that.

* Formalize the concept of Local Node

Also some minor refactoring while creating the dummy placement

* Write intermediate results locally when the results are only needed locally

Before this commit, Citus used to always broadcast all the intermediate
results to remote nodes. However, it is possible to skip pushing
the results to remote nodes always.

There are two notable cases for doing that:

   (a) When the query consists of only intermediate results
   (b) When the query is a zero shard query

In both of the above cases, we don't need to access any data on the shards. So,
it is a valuable optimization to skip pushing the results to remote nodes.

The pattern mentioned in (a) is actually a common patterns that Citus users
use in practice. For example, if you have the following query:

WITH cte_1 AS (...), cte_2 AS (....), ... cte_n (...)
SELECT ... FROM cte_1 JOIN cte_2 .... JOIN cte_n ...;

The final query could be operating only on intermediate results. With this patch,
the intermediate results of the ctes are not unnecessarily pushed to remote
nodes.

* Add specific regression tests

As there are edge cases in Citus MX and with round-robin policy,
use the same queries on those cases as well.

* Fix failure tests

By forcing not to use local execution for intermediate results since
all the tests expects the results to be pushed remotely.

* Fix flaky test

* Apply code-review feedback

Mostly style changes

* Limit the max value of pg_dist_node_seq to reserve for internal use
2020-01-23 18:28:34 +01: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
Onder Kalaci 2f0ef8bc36 Apply feedback 1 2020-01-17 16:06:04 +01:00
Onder Kalaci 0bf1e81e33 Cache local plans on BeginScan 2020-01-17 16:02:57 +01:00
Onder Kalaci 5dc454cdad Exclude localPlannedStatements from copy distributedPlan 2020-01-17 16:02:57 +01:00
Jelte Fennema 246435be7e
Lazy query deparsing executable queries (#3350)
Deparsing and parsing a query can be heavy on CPU. When locally executing 
the query we don't need to do this in theory most of the time.

This PR is the first step in allowing to skip deparsing and parsing
the query in these cases, by lazily creating the query string and
storing the query in the task. Future commits will make use of this and
not deparse and parse the query anymore, but use the one from the task
directly.
2020-01-17 11:49:43 +01:00
Hadi Moshayedi 6cf1c01660 Don't use repartitioned INSERT/SELECT for repartition joins 2020-01-16 23:40:31 -08:00
Hadi Moshayedi 5eeb07124f Repartitioned INSERT/SELECT: include job id in result id prefix 2020-01-16 23:24:52 -08:00
Hadi Moshayedi a079278b0c Repartitioned INSERT/SELECT: Add a GUC to enable/disable it 2020-01-16 23:24:52 -08:00
Hadi Moshayedi ce5eea4885 INSERT/SELECT: make SELECT column names unique 2020-01-16 23:24:52 -08:00
Hadi Moshayedi 97072c9eb1 INSERT/SELECT: show method in EXPLAIN output 2020-01-16 23:24:52 -08:00
Hadi Moshayedi fe548b762f Repartitioned INSERT/SELECT: Test CTEs 2020-01-16 23:24:52 -08:00
Hadi Moshayedi 494cc383cc Repartitioned INSERT/SELECT: Enable RETURNING 2020-01-16 23:24:52 -08:00
Hadi Moshayedi 44a2aede16 Don't start a coordinated transaction on workers.
Otherwise transaction hooks of Citus kick in and might cause unwanted errors.
2020-01-16 23:24:52 -08:00
Hadi Moshayedi 42c3c03b85 Handle extra columns added in ExpandWorkerTargetEntry() in repartitioned INSERT/SELECT 2020-01-16 23:24:52 -08:00
Hadi Moshayedi 89463f9760 Repartitioned INSERT/SELECT: cast columns in SELECT targets 2020-01-16 23:24:52 -08:00
Hadi Moshayedi d67a384350 Enable repartitioned INSERT/SELECT ON CONFLICT. 2020-01-16 23:24:52 -08:00
Hadi Moshayedi b4e5f4b10a Implement INSERT ... SELECT with repartitioning 2020-01-16 23:24:52 -08:00
Hadi Moshayedi ced876358d INSERT/SELECT: Refactor out AddInsertSelectCasts 2020-01-16 23:24:52 -08:00
Hadi Moshayedi d449c1857c INSERT/SELECT: Use ExecutePlan* instead of ExecuteSelect* 2020-01-16 23:24:52 -08:00
Marco Slot 82f1fffa28 Fix epoll_ctl() error message on connection error 2020-01-16 06:40:57 +01:00
Onder Kalaci dc17c2658e Defer shard pruning for fast-path router queries to execution
This is purely to enable better performance with prepared statements.
Before this commit, the fast path queries with prepared statements
where the distribution key includes a parameter always went through
distributed planning. After this change, we only go through distributed
planning on the first 5 executions.
2020-01-16 16:59:36 +01:00
Philip Dubé 4989c9a15c PlacementExecutionDone: We may mark placements as failed multiple times, but should only act the first time. 2020-01-15 18:20:01 +00:00
Marco Slot f1a0582973 Make ApplyLogRedaction a macro and redefine ereport 2020-01-13 18:24:36 +01:00
Marco Slot 06709ee108 Always use NOTICE in log_remote_commands and avoid redaction when possible 2020-01-13 18:24:36 +01:00
Philip Dubé 4b5d6c3ebe Rename RelayFileState to ShardState
Replace FILE_ prefix with SHARD_STATE_
2020-01-12 05:57:53 +00:00
Hadi Moshayedi 527d7d41c1 Implement RedistributeTaskListResult 2020-01-09 23:47:25 -08:00
Hadi Moshayedi c7c460e843 PartitionTasklistResults: Use different queries per placement
We need to know which placement succeeded in executing the worker_partition_query_result() call. Otherwise we wouldn't know which node to fetch from. This change allows that by introducing Task::perPlacementQueryStrings.
2020-01-09 10:55:58 -08:00
Hadi Moshayedi f38d0e5b3f Partitioned task list results. 2020-01-09 10:32:58 -08:00
Onder Kalaci c8f14c9f6c Make sure to update shard states of partitions on failures
Fixes #3331

In #2389, we've implemented support for partitioned tables with rep > 1.
The implementation is limiting the use of modification queries on the
partitions. In fact, we error out when any partition is modified via
EnsurePartitionTableNotReplicated().

However, we seem to forgot an important case, where the parent table's
partition is marked as INVALID. In that case, at least one of the partition
becomes INVALID. However, we do not mark partitions as INVALID ever.

If the user queries the partition table directly, Citus could happily send
the query to INVALID placements -- which are not marked as INVALID.

This PR fixes it by marking the placements of the partitions as INVALID
as well.

The shard placement repair logic already re-creates all the partitions,
so should be fine in that front.
2020-01-06 12:26:08 +01:00
Hadi Moshayedi d7aea7fa10 Implement partitioned intermediate results. 2019-12-24 03:53:39 -08:00
Marco Slot b37ef0e394 Fix error in distributed queries when shards are on the coordinator 2019-12-24 06:36:43 +01:00
Hadi Moshayedi 08eb0ade31 Fix reindent version inconsistencies.
Different versions of reindent tool reformatted citus_custom_scan.c
and citus_copyfuncs.c differently. So some developers spent some
extra attention not to commit these two files after reindent.

This PR tries to address this.
2019-12-19 23:10:34 -08:00
Hadi Moshayedi ef487e0792 Implement fetch_intermediate_results 2019-12-18 10:46:35 -08:00
Hadi Moshayedi 113bd1e5f1 Implement read_intermediate_results 2019-12-17 13:51:16 -08: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
Marco Slot f4031dd477 Clean up transaction block usage logic in adaptive executor 2019-12-17 10:48:19 +01:00
Nils Dijk bfc3d2eb90
make sure to correctly decrement ExecutorLevel (#3311)
DESCRIPTION: Fix counter that keeps track of internal depth in executor

While reviewing #3302 I ran into the `ExecutorLevel` variable which used a variable to keep the original value to restore on successful exit. I haven't explored the full space and if it is possible to get into an inconsistent state. However using `PG_TRY`/`PG_CATCH` seems generally more correct.

Given very bad things will happen if this level is not reset, I kept the failsafe of setting the variiable back to 0 on the `XactCallback` but I did add an assert to treat it as a developer bug.
2019-12-16 20:50:13 +01:00
SaitTalhaNisanci 2829c601dd
replace Begin words in coordinated transactions with use (#3293) 2019-12-16 10:40:31 +03:00
SaitTalhaNisanci 2c040d2c8f
use a function for duplicate code in connection state machine (#3209) 2019-12-12 17:55:38 +03:00
SaitTalhaNisanci a0fe8646e0
add IsHoldOffCancellationReceived utility function (#3290) 2019-12-12 17:32:59 +03:00
SaitTalhaNisanci 053fe18404
not continue in sequential execution if a cancellation is received (#3289) 2019-12-12 17:22:30 +03:00
Hadi Moshayedi 939d3c955b Don't plan function joins locally 2019-12-11 16:53:29 -08:00
Hadi Moshayedi e3e174f30f Fix the way we check for local/reference table joins in the executor 2019-12-11 12:50:20 -08:00
SaitTalhaNisanci 13204487e9
remove copyright years (#3286) 2019-12-11 21:14:08 +03:00
SaitTalhaNisanci d10f97998c rename REMOTE_TRANS_INVALID to REMOTE_TRANS_NOT_STARTED 2019-12-11 15:24:18 +03:00
Marco Slot 133b8e1e0e Move coordinator insert..select logic into executor 2019-12-10 11:21:35 -08:00
Hadi Moshayedi d28beb3711 Detect SQL UDF Calls. 2019-12-05 14:31:05 -08:00
Philip Dubé 5a17fd6d9d Test more reference/local cases, also ALTER ROLE
Test ALTER ROLE doesn't deadlock when coordinator added, or propagate from mx workers

Consolidate wait_until_metadata_sync & verify_metadata to multi_test_helpers
2019-12-03 22:23:14 +00:00
Marco Slot b1b13e394e Fix segfault when executing DDL via UDF 2019-12-01 22:54:41 +01:00
Marco Slot 16d1ad3666 Remove distinction between SQL_TASK and ROUTER_TASK 2019-11-29 05:58:29 +01:00
SaitTalhaNisanci aeec3d1544
fix typo in dependent jobs and dependent task (#3244) 2019-11-28 23:47:28 +03:00
Philip Dubé 261a9de42d Fix typos:
VAR_SET_VALUE_KIND -> VAR_SET_VALUE kind
beginnig -> beginning
plannig -> planning
the the -> the
er then -> er than
2019-11-25 23:24:13 +00: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
Hanefi Onaldi 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.
2019-11-20 15:26:36 +03:00
Hadi Moshayedi d9dcba25e3 Plan reference/local table joins locally 2019-11-15 07:36:50 -08:00
Onder Kalaci 90943a6ce6 Do not include coordinator shards when round-robin is selected
When the user picks "round-robin" policy, the aim is that the load
is distributed across nodes. However, for reference tables on the
coordinator, since local execution kicks in immediately, round-robin
is ignored.

With this change, we're excluding the placement on the coordinator.
Although the approach seems a little bit invasive because of
modifications in the placement list, that sounds acceptable.

We could have done this in some other ways such as:

1) Add a field to "Task->roundRobinPlacement" (or such), which is
updated as the first element after RoundRobinPolicy is applied.
During the execution, if that placement is local to the coordinator,
skip it and try the other remote placements.

2) On TaskAccessesLocalNode()@local_execution.c, check
task_assignment_policy, if round-robin selected and there is local
placement on the coordinator, skip it. However, task assignment is done
on planning, but this decision is happening on the execution, which
could create weird edge cases.
2019-11-15 06:03:32 -08:00
Hadi Moshayedi 15af1637aa Replicate reference tables to coordinator. 2019-11-15 05:50:19 -08: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ı 0b3d4e55d9
Local execution should not change hasReturning for distributed tables (#3160)
It looks like the logic to prevent RETURNING in reference tables to
have duplicate entries that comes from local and remote executions
leads to missing some tuples for distributed tables.

With this PR, we're ensuring to kick in the logic for reference tables
only.
2019-11-08 12:49:56 +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 7c410e3cd7
pass CitusCustomState directly to adaptive executor (#3151) 2019-11-01 19:57:32 +03:00
SaitTalhaNisanci c7ceca3216
update outdated comment in JobExecutorType (#3148) 2019-11-01 11:36:56 +03:00
SaitTalhaNisanci 70e46703aa
Fix debug1 message in JobExecutorType (#3147)
When citus.enable_repartition_joins guc is set to on, and we have
adaptive executor, there was a typo in the debug message, which was
saying realtime executor no adaptive executor.
2019-11-01 11:14:19 +03:00
SaitTalhaNisanci 29d45bd1b9
Do not assign InvalidOid for local execution while extracting parameters (#3131)
* do not assign InvalidOid for local execution while extracting parameters

* rename functions

* rename parameter and replace function
2019-10-28 14:28:22 +03:00
Önder Kalacı dceaddbe4d
Remove real-time/router executors (step 1) (#3125)
See #3125 for details on each item.

* Remove real-time/router executor tests-1

These are the ones which doesn't have '_%d' in the test
output files.

* Remove real-time/router executor tests-2

These are the ones which has in the test
output files.

* Move the tests outputs to correct place

* Make sure that single shard commits use 2PC on adaptive executor

It looks like we've messed the tests in #2891. Fixing back.

* Use adaptive executor for all router queries

This becomes important because when task-tracker is picked, we
used to pick router executor, which doesn't make sense.

* Remove explicit references to real-time/router executors in the tests

* JobExecutorType never picks real-time/router executors

* Make sure to go incremental in test output numbers

* Even users cannot pick real-time anymore

* Do not use real-time/router custom scans

* Get rid of unnecessary normalizations

* Reflect unneeded normalizations

* Get rid of unnecessary test output file
2019-10-25 10:54:54 +02:00
Onder Kalaci a208f8b151 Fix memory leak on ReceiveResults
It turns out that TupleDescGetAttInMetadata() allocates quite a lot
of memory. And, if the target list is long and there are too many rows
returning, the leak becomes appereant.

You can reproduce the issue wout the fix with the following commands:

```SQL

CREATE TABLE users_table (user_id int, time timestamp, value_1 int, value_2 int, value_3 float, value_4 bigint);
SELECT create_distributed_table('users_table', 'user_id');

insert into users_table SELECT i, now(), i, i, i, i FROM generate_series(0,99999)i;

-- load faster

-- 200,000
INSERT INTO users_table SELECT * FROM users_table;

-- 400,000
INSERT INTO users_table SELECT * FROM users_table;

-- 800,000
INSERT INTO users_table SELECT * FROM users_table;

-- 1,600,000
INSERT INTO users_table SELECT * FROM users_table;

-- 3,200,000
INSERT INTO users_table SELECT * FROM users_table;

-- 6,400,000
INSERT INTO users_table SELECT * FROM users_table;

-- 12,800,000
INSERT INTO users_table SELECT * FROM users_table;

-- making the target list entry wider speeds up the leak to show up
 select *,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,*,* FROM users_table ;

 ```
2019-10-22 17:22:26 +02:00
Jelte Fennema 7abedc38b0
Support subqueries in HAVING (#3098)
Areas for further optimization:
- Don't save subquery results to a local file on the coordinator when the subquery is not in the having clause
- Push the the HAVING with subquery to the workers if there's a group by on the distribution column
- Don't push down the results to the workers when we don't push down the HAVING clause, only the coordinator needs it

Fixes #520
Fixes #756
Closes #2047
2019-10-16 16:40:14 +02: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é 74cb168205 Remove Postgres 10 support 2019-10-11 21:56:56 +00:00
Hadi Moshayedi 5e97e5c98e Don't push down queries when in subqueries/ctes 2019-09-30 14:22:05 -07:00
Marco Slot 2868e02a3d Implement SELECT function call delegation.
When a function is marked as colocated with a distributed table,
we try delegating queries of kind "SELECT func(...)" to workers.

We currently only support this simple form, and don't delegate
forms like "SELECT f1(...), f2(...)", "SELECT f1(...) FROM ...",
or function calls inside transactions.

As a side effect, we also fix the transactional semantics of DO blocks.
Previously we didn't consider a DO block a multi-statement transaction.
Now we do.

Co-authored-by: Marco Slot <marco@citusdata.com>
Co-authored-by: serprex <serprex@users.noreply.github.com>
Co-authored-by: pykello <hadi.moshayedi@microsoft.com>
2019-09-27 09:13:25 -07:00
Marco Slot e58d76c5f6 Fix assert failure in bare SELECT FROM reference table FOR UPDATE in MX 2019-09-23 17:00:09 +02:00
Philip Dubé 492d1b2cba ActivePrimaryNodeList: add lockMode parameter 2019-09-13 17:44:56 +00:00
Philip Dubé 5e5f4628a0 Fix pg12 compile 2019-09-13 17:25:30 +00:00
Nils Dijk 2879689441
Distribute Types to worker nodes (#2893)
DESCRIPTION: Distribute Types to worker nodes

When to propagate
==============

There are two logical moments that types could be distributed to the worker nodes
 - When they get used ( just in time distribution )
 - When they get created ( proactive distribution )

The just in time distribution follows the model used by how schema's get created right before we are going to create a table in that schema, for types this would be when the table uses a type as its column.

The proactive distribution is suitable for situations where it is benificial to have the type on the worker nodes directly. They can later on be used in queries where an intermediate result gets created with a cast to this type.

Just in time creation is always the last resort, you cannot create a distributed table before the type gets created. A good example use case is; you have an existing postgres server that needs to scale out. By adding the citus extension, add some nodes to the cluster, and distribute the table. The type got created before citus existed. There was no moment where citus could have propagated the creation of a type.

Proactive is almost always a good option. Types are not resource intensive objects, there is no performance overhead of having 100's of types. If you want to use them in a query to represent an intermediate result (which happens in our test suite) they just work.

There is however a moment when proactive type distribution is not beneficial; in transactions where the type is used in a distributed table.

Lets assume the following transaction:

```sql
BEGIN;
CREATE TYPE tt1 AS (a int, b int);
CREATE TABLE t1 AS (a int PRIMARY KEY, b tt1);
SELECT create_distributed_table('t1', 'a');
\copy t1 FROM bigdata.csv
```

Types are node scoped objects; meaning the type exists once per worker. Shards however have best performance when they are created over their own connection. For the type to be visible on all connections it needs to be created and committed before we try to create the shards. Here the just in time situation is most beneficial and follows how we create schema's on the workers. Outside of a transaction block we will just use 1 connection to propagate the creation.

How propagation works
=================

Just in time
-----------

Just in time propagation hooks into the infrastructure introduced in #2882. It adds types as a supported object in `SupportedDependencyByCitus`. This will make sure that any object being distributed by citus that depends on types will now cascade into types. When types are depending them self on other objects they will get created first.

Creation later works by getting the ddl commands to create the object by its `ObjectAddress` in `GetDependencyCreateDDLCommands` which will dispatch types to `CreateTypeDDLCommandsIdempotent`.

For the correct walking of the graph we follow array types, when later asked for the ddl commands for array types we return `NIL` (empty list) which makes that the object will not be recorded as distributed, (its an internal type, dependant on the user type).

Proactive distribution
---------------------

When the user creates a type (composite or enum) we will have a hook running in `multi_ProcessUtility` after the command has been applied locally. Running after running locally makes that we already have an `ObjectAddress` for the type. This is required to mark the type as being distributed.

Keeping the type up to date
====================

For types that are recorded in `pg_dist_object` (eg. `IsObjectDistributed` returns true for the `ObjectAddress`) we will intercept the utility commands that alter the type.
 - `AlterTableStmt` with `relkind` set to `OBJECT_TYPE` encapsulate changes to the fields of a composite type.
 - `DropStmt` with removeType set to `OBJECT_TYPE` encapsulate `DROP TYPE`.
 - `AlterEnumStmt` encapsulates changes to enum values.
    Enum types can not be changed transactionally. When the execution on a worker fails a warning will be shown to the user the propagation was incomplete due to worker communication failure. An idempotent command is shown for the user to re-execute when the worker communication is fixed.

Keeping types up to date is done via the executor. Before the statement is executed locally we create a plan on how to apply it on the workers. This plan is executed after we have applied the statement locally.

All changes to types need to be done in the same transaction for types that have already been distributed and will fail with an error if parallel queries have already been executed in the same transaction. Much like foreign keys to reference tables.
2019-09-13 17:46:07 +02:00
Onder Kalaci 0b0c779c77 Introduce the concept of Local Execution
/*
 * local_executor.c
 *
 * The scope of the local execution is locally executing the queries on the
 * shards. In other words, local execution does not deal with any local tables
 * that are not shards on the node that the query is being executed. In that sense,
 * the local executor is only triggered if the node has both the metadata and the
 * shards (e.g., only Citus MX worker nodes).
 *
 * The goal of the local execution is to skip the unnecessary network round-trip
 * happening on the node itself. Instead, identify the locally executable tasks and
 * simply call PostgreSQL's planner and executor.
 *
 * The local executor is an extension of the adaptive executor. So, the executor uses
 * adaptive executor's custom scan nodes.
 *
 * One thing to note that Citus MX is only supported with replication factor = 1, so
 * keep that in mind while continuing the comments below.
 *
 * On the high level, there are 3 slightly different ways of utilizing local execution:
 *
 * (1) Execution of local single shard queries of a distributed table
 *
 *      This is the simplest case. The executor kicks at the start of the adaptive
 *      executor, and since the query is only a single task the execution finishes
 *      without going to the network at all.
 *
 *      Even if there is a transaction block (or recursively planned CTEs), as long
 *      as the queries hit the shards on the same, the local execution will kick in.
 *
 * (2) Execution of local single queries and remote multi-shard queries
 *
 *      The rule is simple. If a transaction block starts with a local query execution,
 *      all the other queries in the same transaction block that touch any local shard
 *      have to use the local execution. Although this sounds restrictive, we prefer to
 *      implement in this way, otherwise we'd end-up with as complex scenarious as we
 *      have in the connection managements due to foreign keys.
 *
 *      See the following example:
 *      BEGIN;
 *          -- assume that the query is executed locally
 *          SELECT count(*) FROM test WHERE key = 1;
 *
 *          -- at this point, all the shards that reside on the
 *          -- node is executed locally one-by-one. After those finishes
 *          -- the remaining tasks are handled by adaptive executor
 *          SELECT count(*) FROM test;
 *
 *
 * (3) Modifications of reference tables
 *
 *		Modifications to reference tables have to be executed on all nodes. So, after the
 *		local execution, the adaptive executor keeps continuing the execution on the other
 *		nodes.
 *
 *		Note that for read-only queries, after the local execution, there is no need to
 *		kick in adaptive executor.
 *
 *  There are also few limitations/trade-offs that is worth mentioning. First, the
 *  local execution on multiple shards might be slow because the execution has to
 *  happen one task at a time (e.g., no parallelism). Second, if a transaction
 *  block/CTE starts with a multi-shard command, we do not use local query execution
 *  since local execution is sequential. Basically, we do not want to lose parallelism
 *  across local tasks by switching to local execution. Third, the local execution
 *  currently only supports queries. In other words, any utility commands like TRUNCATE,
 *  fails if the command is executed after a local execution inside a transaction block.
 *  Forth, the local execution cannot be mixed with the executors other than adaptive,
 *  namely task-tracker, real-time and router executors. Finally, related with the
 *  previous item, COPY command cannot be mixed with local execution in a transaction.
 *  The implication of that any part of INSERT..SELECT via coordinator cannot happen
 *  via the local execution.
 */
2019-09-12 11:51:25 +02:00
Onder Kalaci 485189c0b6 Make sure that lost connections are handled properly
Before this patch, when a connection is lost, we'd have the following
situation:

    - Pop a task execution from readyQueue
    - Lost connection
    - Fail the session/pool. -> This step was not acting properly
      because we've popped the task, but not set to session->currentTask
      yet

After the patch:

    - Pop a task execution from readyQueue
    - Immediately set it to session->currentTask
    - Lost connection
    - Fail the session/pool. -> At this step, failing the
      session would trigger query failures (or failovers)
      properly.
2019-09-10 17:54:27 +02:00
Jelte Fennema cbecf97c84
Move tuplestore setup to a helper function (#2898)
* Add tuplestore helpers

* More detailed error messages in tuplestore

* Add CreateTupleDescCopy to SetupTuplestore

* Use new SetupTuplestore helper function

* Remove unnecessary copy

* Remove comment about undefined behaviour
2019-08-27 09:11:08 +02:00
Philip Dubé fe10ca453d Implement FileCompat to abstract pg12 requiring API consumer to track file offsets 2019-08-22 18:57:47 +00: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é db5a7f49a7 Task Tracker: fix error being copy pasted from above block 2019-08-21 15:44:01 +00:00
Philip Dubé f4e513b3d4 Introduce citus.single_shard_commit_protocol for if users want 1PC on writes to replicas 2019-08-15 18:49:40 +00:00
Hadi Moshayedi 009d8b7401 Some cleanup 2019-08-12 15:38:52 -07:00
Onder Kalaci 35ee896f3d Get rid of an unnecessary parameter
targetPoolSize parameter for ExecuteUtilityTaskListWithoutResults
becomes obsolete, just remove it.
2019-08-07 19:35:56 +02:00
Onder Kalaci b2e01d0745 Refactor switching to sequential mode
We don't need to wait until the execution. As soon as we realize
that we need sequential execution, we should do it.
2019-08-07 19:35:56 +02:00
Philip Dubé fdc0ef6392 Adaptive executor: use 2PC when replication_factor > 1 2019-08-01 23:55:12 +00:00
Philip Dubé 064bd66a20 Avoid segfault in logging queries 2019-07-31 15:28:46 +00:00
Marco Slot e2bc09838e Use ereport instead of elog in adaptive executor 2019-07-23 20:40:32 +02:00
Marco Slot bd111366b0 Skip CheckConnectionTimeout when checkForPoolTimeout is false 2019-07-23 20:40:32 +02:00
Marco Slot a3811b1e55 Avoid FindWorkerNode calls in adaptive executor 2019-07-23 20:40:32 +02:00
Marco Slot 4444d92dbc Set initial pool size to cached connection count 2019-07-23 20:40:32 +02:00
Marco Slot 4c0c33365e Avoid creating a redundant event set at the start 2019-07-23 20:40:32 +02:00
Marco Slot 32e7a80960 Avoid unnecessary calls to PQconsumeInput 2019-07-23 20:40:32 +02:00
Marco Slot 71ad5c095b Use ModifyWaitEvent when only wait flags changed 2019-07-23 20:40:32 +02: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
Hadi Moshayedi 805a2ac602 Fix a use after free in adaptive executor 2019-07-02 10:12:13 -07:00
Marco Slot d6c667946c Fix citus_executor_name mapping by reimplementing it in C 2019-06-29 22:38:29 +02:00
Marco Slot 70c0d96507 Track partition key for adaptive executor in CitusEndScan 2019-06-29 21:37:15 +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
Philip Dubé 5c62f9935a Router planner: reject SELECT FOR UPDATE ctes 2019-06-26 10:32:01 +02:00
Onder Kalaci ad93d6feea Change the order of placement access added to the list
This is to make sure that the error messages related to foreign keys
to reference tables shows the exact placement access name instead of
SELECT.
2019-06-23 11:32:58 +02:00
Hadi Moshayedi 4bbae02778 Make COPY compatible with unified executor. 2019-06-20 19:53:40 +02:00
Hadi Moshayedi 2e6d04df7b Refactor ExecuteModifyTasksSequentially. 2019-06-20 18:38:57 +02:00
Philip Dubé 4bfcf5b665 Enable Werror for all warnings
Changes to ruleutils match changes made upstream to silence gcc fallthrough warnings
2019-06-18 14:43:54 -07:00
Hadi Moshayedi 85325e0098 Refactor ScanStateGetExecutorState into its own function. 2019-06-05 09:16:43 -07:00
Hadi Moshayedi 0b01c59fa6 Refactor ScanStateGetTupleDescriptor() into a function. 2019-06-04 15:19:49 -07:00
Marco Slot bb3a96eacb Cache a configurable number of connections at xact end 2019-05-29 13:24:31 +02:00
Hadi Moshayedi 8ae47e1244 Fix comments for RemoteFileDestReceiverStartup and CitusCopyDestReceiverStartup 2019-05-21 09:03:22 -07:00
Hadi Moshayedi b5c0ca45f1 Remove stopOnFailure flag from EndRemoteCopy() 2019-05-11 06:18:34 -07:00
Hadi Moshayedi e584961267 Fix mixed declarations and code warnings 2019-05-08 12:51:40 -07:00
Onder Kalaci 495b6e9b62 Refactor Parallel Relation Access Recording
Instead of scattering the code around, we move all the
logic into a single function.

This will help supporting foreign keys to reference tables
in the unified executor with a single line of change, just
calling this function.
2019-05-02 18:12:33 +03:00
Hadi Moshayedi 32ecb6884c Test ROLLBACK TO SAVEPOINT with multi-shard CTE failures 2019-05-01 09:33:43 -07:00
Hadi Moshayedi b69a762e0b Fix savepoint rollback after multi-shard update failure. 2019-05-01 09:33:43 -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
Onder Kalaci 004f28e18c Sort output of RETURNING
The feature is only intended for getting consistent outputs for the regression tests.

RETURNING does not have any ordering gurantees and with unified executor, the ordering
of query executions on the shards are also becoming unpredictable. Thus, we're enforcing
ordering when a GUC is set.

We implicitly add an `ORDER BY` something equivalent of
	`
	  RETURNING expr1, expr2, .. ,exprN
	  ORDER BY expr1, expr2, .. ,exprN
	`

As described in the code comments as well, this is probably not the most
performant approach we could implement. However, since we're only
targeting regression tests, I don't see any issues with that. If we
decide to expand this to a feature to users, we should revisit the
implementation and improve the performance.
2019-04-24 11:51:19 +03:00
Marco Slot e3b7e74f43 Allow rescan in DECLARE .. WITH HOLD 2019-03-22 11:25:55 +01:00
Marco Slot 5ff1821411 Cache the current database name
Purely for performance reasons.
2019-03-20 12:14:46 +03:00
Marco Slot f2abf2b8e5 Functions are treated as transaction blocks 2019-03-15 16:34:08 -06:00
Hadi Moshayedi a9e6d06a98 Skip execution of ALTER TABLE constraint checks on the coordinator 2019-03-14 15:40:56 -07:00
Hadi Moshayedi f4d3b94e22
Fix some of the casts for groupId (#2609)
A small change which partially addresses #2608.
2019-03-05 12:06:44 -08:00
Jason Petersen 339e6e661e
Remove 9.6 (#2554)
Removes support and code for PostgreSQL 9.6

cr: @velioglu
2019-01-16 13:11:24 -07:00
Marco Slot 1b1c6374f7
Execute CREATE INDEX CONCURRENTLY concurrently 2018-12-21 14:02:59 -07: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
Marco Slot aff37cf1bc Control multi-shard modify locks with enable_deadlock_prevention 2018-11-28 02:59:50 +01:00
Marco Slot 6aa5592e52 Add user ID suffix to intermediate files in re-partition jobs 2018-11-23 08:36:11 +01:00
Marco Slot a59bf31c76 Use worker_execute_sql_task UDF in task-tracker executor 2018-11-22 18:15:33 +01:00
Marco Slot caf402d506 COPY to a task file no longer switches to superuser 2018-11-22 18:15:33 +01:00
Marco Slot f383e4f307
Description: Refactor code that handles DDL commands from one file into a module
The file handling the utility functions (DDL) for citus organically grew over time and became unreasonably large. This refactor takes that file and refactored the functionality into separate files per command. Initially modeled after the directory and file layout that can be found in postgres.

Although the size of the change is quite big there are barely any code changes. Only one two functions have been added for readability purposes:

- PostProcessIndexStmt which is extracted from PostProcessUtility
- PostProcessAlterTableStmt which is extracted from multi_ProcessUtility

A README.md has been added to `src/backend/distributed/commands` describing the contents of the module and every file in the module.
We need more documentation around the overloading of the COPY command, for now the boilerplate has been added for people with better knowledge to fill out.
2018-11-14 13:36:27 +01:00
Onder Kalaci 6e05921736 Processes that are blocked on advisory locks show up in wait edges
Assign the distributed transaction id before trying to acquire the
executor advisory locks. This is useful to show this backend in citus
lock graphs (e.g., dump_global_wait_edges() and citus_lock_waits).
2018-10-24 13:32:13 +03:00
Hadi Moshayedi 3e00bf1c0d Don't throw error for DROP DATABASE IF EXISTS 2018-10-23 09:45:03 -04:00
Marco Slot d56baefe3d Allow simple DML commands from hot standby 2018-10-06 10:54:44 +02:00
velioglu 512d23934f Show router modify,select and real-time queries on MX views 2018-10-02 13:59:38 +03:00
Onder Kalaci cdc0d1491c Make sure to use correct execution mode for TRUNCATE
We used to set the execution mode in the truncate trigger. However,
when multiple tables are truncated with a single command, we could
set the execution mode very late. Instead, now set the execution mode
on the utility hook.
2018-09-25 15:35:27 +03:00
Onder Kalaci abc443d7fa Make sure that shard repair considers replication factor 2018-09-21 15:24:49 +03:00
Onder Kalaci c1b5a04f6e Allow partitioned tables with replication factor > 1
With this commit, we all partitioned distributed tables with
replication factor > 1. However, we also have many restrictions.

In summary, we disallow all kinds of modifications (including DDLs)
on the partition tables. Instead, the user is allowed to run the
modifications over the parent table.

The necessity for such a restriction have two aspects:
   - We need to acquire shard resource locks appropriately
   - We need to handle marking partitions INVALID in case
     of any failures. Note that, in theory, the parent table
     should also become INVALID, which is too aggressive.
2018-09-21 14:40:41 +03:00
Murat Tuncer b6930e3db9 Add distributed locking to truncated mx tables
We acquire distributed lock on all mx nodes for truncated
tables before actually doing truncate operation.

This is needed for distributed serialization of the truncate
command without causing a deadlock.
2018-09-21 14:23:19 +03:00
Murat Tuncer 0f6e514bfb Fixes a bug on not being able to drop index on a partitioned table.
Reason for the failure is that PG11 introduced a new relation kind
RELKIND_PARTITIONED_INDEX to be used for partitioned indices.

We expanded our check to cover that case.
2018-09-19 13:15:05 +03:00
Marco Slot f34ab55389 Fix bug preventing rollback in stored procedure 2018-08-31 20:49:20 +02:00
velioglu bd30e3e908 Add support for writing to reference tables from MX nodes 2018-08-27 18:15:04 +03:00
Onder Kalaci b8af8c359b Make sure that modifying CTEs always use the correct execution mode 2018-08-23 14:53:55 +03:00
mehmet furkan şahin ef9f38b68d ApplyLogRedaction noop func is added 2018-08-17 14:48:54 -07:00
Onder Kalaci 85d418412d Fix DDL execution problem on MX when search_path is used
Make sure that the coordinator sends the commands when the search
path synchronised with the coordinator's search_path. This is only
important when Citus sends the commands that are directly relayed
to the worker nodes. For example, the deparsed DLL commands or
queries always adds schema qualifications to the queries. So, they
do not require this change.
2018-08-13 16:34:50 +03:00
velioglu e23625bf5e Use contype to check for FK constraint instead of reading catalog table 2018-07-24 15:53:05 +03:00
mehmet furkan şahin 6d0fbbace7 ALTER TABLE %s ADD COLUMN constraint check is added 2018-07-24 15:53:05 +03:00
Marco Slot 625816242a
Don't try to check unopened connection in EXEC_TASK_FAILED state 2018-07-23 11:41:02 -06:00
Nils Dijk df98900f80
fix missing space for tablein in error 2018-07-20 15:05:13 +02:00
Marco Slot 89870e76ce Add a select_opens_transaction_block GUC 2018-07-08 03:50:39 +02:00
Onder Kalaci 7fb529aab9 Some stylistic improvements in the foreign keys to reference table
changes.
2018-07-05 23:23:34 +03:00
Nils Dijk c1c8c38dc9 create placeholder for policy ddl 2018-07-05 11:07:01 +02:00
Murat Tuncer 901066a421 Move partition key logging related code from enterprise 2018-07-04 13:11:34 +03:00
Onder Kalaci d83be3a33f Enforce foreign key restrictions inside transaction blocks
When a hash distributed table have a foreign key to a reference
table, there are few restrictions we have to apply in order to
prevent distributed deadlocks or reading wrong results.

The necessity to apply the restrictions arise from cascading
nature of foreign keys. When a foreign key on a reference table
cascades to a distributed table, a single operation over a single
connection can acquire locks on multiple shards of the distributed
table. Thus, any parallel operation on that distributed table, in the
same transaction should not open parallel connections to the shards.
Otherwise, we'd either end-up with a self-distributed deadlock or
read wrong results.

As briefly described above, the restrictions that we apply is done
by tracking the distributed/reference relation accesses inside
transaction blocks, and act accordingly when necessary.

The two main rules are as follows:
   - Whenever a parallel distributed relation access conflicts
     with a consecutive reference relation access, Citus errors
     out
   - Whenever a reference relation access is followed by a
     conflicting parallel relation access, the execution mode
     is switched to sequential mode.

There are also some other notes to mention:
   - If the user does SET LOCAL citus.multi_shard_modify_mode
     TO 'sequential';, all the queries should simply work with
     using one connection per worker and sequentially executing
     the commands. That's obviously a slower approach than Citus'
     usual parallel execution. However, we've at least have a way
     to run all commands successfully.

   - If an unrelated parallel query executed on any distributed
     table, we cannot switch to sequential mode. Because, the essense
     of sequential mode is using one connection per worker. However,
     in the presence of a parallel connection, the connection manager
     picks those connections to execute the commands. That contradicts
     with our purpose, thus we error out.

   - COPY to a distributed table cannot be executed in sequential mode.
     Thus, if we switch to sequential mode and COPY is executed, the
     operation fails and there is currently no way of implementing that.
     Note that, when the local table is not empty and create_distributed_table
     is used, citus uses COPY internally. Thus, in those cases,
     create_distributed_table() will also fail.

   - There is a GUC called citus.enforce_foreign_key_restrictions
     to disable all the checks. We added that GUC since the restrictions
     we apply is sometimes a bit more restrictive than its necessary.
     The user might want to relax those. Similarly, if you don't have
     CASCADEing reference tables, you might consider disabling all the
     checks.
2018-07-03 17:05:55 +03:00
velioglu 6be6911ed9 Create foreign key relation graph and functions to query on it 2018-07-03 17:05:55 +03:00
mehmet furkan şahin 4db72c99f6 Specific DDLs are sequentialized when there is FK
-[x] drop constraint
-[x] drop column
-[x] alter column type
-[x] truncate

are sequentialized if there is a foreign constraint from
a distributed table to a reference table on the affected relations
by the above commands.
2018-07-03 17:05:55 +03:00
mehmet furkan şahin 2fa4e38841 FK from dist to ref can be added with alter table 2018-07-03 17:05:01 +03:00
Murat Tuncer 4d35b92016 Add groundwork for citus_stat_statements api 2018-06-27 14:20:03 +03:00
Brian Cloutier 5ce18327a7 Don't spinloop when trying to cleanup a failed connection 2018-06-26 13:13:34 -07:00
Onder Kalaci 8ccb8b679e Real-time executor marks multi shard relation accesses before opening connections 2018-06-25 18:40:31 +03:00
Onder Kalaci 2890154420 Make sure that TRUNCATE always opens a DDL access 2018-06-25 18:40:31 +03:00
Onder Kalaci 21038f0d0e Make sure that inter-shard DDL commands are always covers both tables 2018-06-25 18:40:30 +03:00
Onder Kalaci 2f01894589 Track relation accesses using the connection management infrastructure 2018-06-25 18:40:30 +03: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
Onder Kalaci 7762d81cba Move test UDF under test folder 2018-06-21 08:42:44 +03:00
Onder Kalaci 8f5821493a Implement C interface for setting GUC
We need the ability to switch to sequential mode (e.g.,
 SET LOCAL citus.multi_shard_modify_mode = 'sequential'). This
commit enables that.
2018-06-19 10:23:43 +03:00
Marco Slot f3f2805978
Fix use-after-free that may occur for INSERT..SELECT in prepared statements 2018-06-18 22:55:06 -06:00
velioglu 53b2e81d01 Adds SELECT ... FOR UPDATE support for router plannable queries 2018-06-18 13:55:17 +03:00
Marco Slot 0bbe778760 Rename failOnError to alwaysThrowErrorOnFailure 2018-06-14 23:37:47 +02:00
Marco Slot 4ab8e87090 Always throw errors on failure on critical connection in router executor 2018-06-14 23:33:07 +02:00
Nils Dijk 73efcb22c4 Extract RoleSpecString and resolve role references 2018-06-14 11:38:42 +02:00
mehmet furkan şahin d1a3b20115 foreign_constraint_utils is created 2018-06-07 18:19:24 +03:00
Onder Kalaci a5370f5bb0 Realtime executor honours multi_shard_modify_mode
We're relying on multi_shard_modify_mode GUC for real-time SELECTs.
The name of the GUC is unfortunate, but, adding one more GUC
(or renaming the GUC) would make the UX even worse. Given that this
mode is mostly important for transaction blocks that involve modification
/DDL queries along with real-time SELECTs, we can live with the confusion.
2018-06-06 14:59:54 +03:00
Onder Kalaci d918556dca INSERT .. SELECT pushdown honors multi_shard_modification_mode 2018-06-06 12:42:23 +03:00
Onder Kalaci 336044f2a8 master_modify_multiple_shards() and TRUNCATE honors multi_shard_modification_mode 2018-06-06 12:29:05 +03:00
Onder Kalaci df44956dc3 Make sure that sequential DDL opens a single connection to each node
After this commit DDL commands honour `citus.multi_shard_modify_mode`.

We preferred using the code-path that executes single task router
queries (e.g., ExecuteSingleModifyTask()) in order not to invent
a new executor that is only applicable for DDL commands that require
sequential execution.
2018-06-05 17:52:17 +03:00
Marco Slot fd4ff29f2f Add a debug message with distribution column value 2018-06-05 15:09:17 +03:00
Murat Tuncer ba50e3f33e Add handling for grant/revoke all tables in schema 2018-05-31 13:47:02 +03:00
Marco Slot 5f5f7b4fe0 Throw an error if placements cannot be found in router executor 2018-05-08 22:39:18 -04:00
Marco Slot 9438e5bde9 Ensure single-shard modifying CTEs are part of distributed transaction 2018-05-06 12:49:40 +02:00
Marco Slot 90cdfff602 Implement recursive planning for DML statements 2018-05-03 14:42:28 +02:00
velioglu 32bcd610c1 Support modify queries with multiple tables
With this commit we begin to support modify queries with multiple
tables if these queries are pushdownable.
2018-05-02 16:22:26 +03:00
Brian Cloutier f8fb7a27fb Don't copyObject into the wrong memory context
utilityStmt sometimes (such as when it's inside of a plpgsql function)
comes from a cached plan, which is kept in a child of the
CacheMemoryContext. When we naively call copyObject we're copying it into
a statement-local context, which corrupts the cached plan when it's
thrown away.
2018-05-01 15:34:32 -07:00
mehmet furkan şahin f2555317b6 ProcessVacuumStmt update on names 2018-04-27 14:37:01 +03:00
mehmet furkan şahin a4153c6ab1 notice handler is implemented 2018-04-27 14:37:01 +03:00
Marco Slot 3d3c19a717
Improve messages for essential connection failures 2018-04-26 12:58:47 -06:00
Hadi Moshayedi 24659a97dc
Fail task in real-time executor if no placements found. (#2133) 2018-04-26 12:05:24 -04:00
Murat Tuncer a6fe5ca183 PG11 compatibility update
- changes in ruleutils_11.c is reflected
- vacuum statement api change is handled. We now allow
  multi-table vacuum commands.
- some other function header changes are reflected
- api conflicts between PG11 and earlier versions
  are handled by adding shims in version_compat.h
- various regression tests are fixed due output and
  functionality in PG1
- no change is made to support new features in PG11
  they need to be handled by new commit
2018-04-26 11:29:43 +03:00
Brian Cloutier a59c1c634e Fix cancellation of real time queries
Without this change multi_real_time_transaction blocks forever (on
Windows) in the block where it repeatedly calls pg_advisory_lock(15).
This happens because the deadlock detector tries to cancel the backend
but the backend never processes that signal.
2018-04-17 14:26:22 -07:00
Burak Yucesoy b33b282030 Fix bug while DROPping partitioned table from worker
We recently added partitionin support to Citus MX. We should not execute
DROP table commands from MX workers but at the moment we try to execute
such commands for partitioned tables. This PR fixes that problem by
adding check.
2018-04-09 13:50:21 +03:00
Burak Yucesoy 0c283fa8a3 Add partitioning support to MX tables
Previously, we prevented creation of partitioned tables on Citus MX.
We decided to not focus on this feature until there is a need. Since
now there are requests for this feature, we are implementing support
for partitioned tables on Citus MX.
2018-04-06 12:47:06 +03:00
velioglu 698d585fb5 Remove broadcast join logic
After this change all the logic related to shard data fetch logic
will be removed. Planner won't plan any ShardFetchTask anymore.
Shard fetch related steps in real time executor and task-tracker
executor have been removed.
2018-03-30 11:45:19 +03:00
Murat Tuncer 224b0a8c14 Replace poll with select/poll
Windows does not have poll(), so fall back to select()
2018-03-21 20:05:00 -07:00
Onder Kalaci 7dc9589b56 Handle failures during I/O
This commit checks the connection status right after any IO happens
on the socket.

This is necessary since before this commit we didn't pass any information
to the higher level functions whether we're done with the connection
(e.g., no IO required anymore) or an errors happened during the IO.
2018-03-02 08:33:53 +02:00
Metin Doslu bcf660475a Add support for modifying CTEs 2018-02-27 15:08:32 +02:00
Marco Slot ee6a751798 Only copy distributed plan when modifying it 2018-02-12 16:30:55 +01:00
Marco Slot bd0ebac865 Skip call to ActiveReadableNodeList when there are no subplans 2018-01-29 16:05:10 +01:00
Hadi Moshayedi ff26bcd5a5
Include sys/stat.h for S_IRUSR and S_IWUSR. (#1977) 2018-01-26 16:21:48 -05:00
Dimitri Fontaine 952da72c55 Implement ALTER TABLE|INDEX ... SET|RESET ().
PostgreSQL implements support for several relation kinds in a single
statement, such as in the AlterTableStmt case, which supports both tables
and indexes and more (see ATExecSetRelOptions in PostgreSQL source code file
src/backend/commands/tablecmds.c for an example of that).

As a consequence, this patch implements support for setting and resetting
storage parameters on both relation kinds.
2018-01-17 21:56:40 +01:00
Dimitri Fontaine 17266e3301 Implement ALTER INDEX ... RENAME TO ...
The command is now distributed among the shards when the table is
distributed. To that effect, we fill in the DDLJob's targetRelationId with
the OID of the table for which the index is defined, rather than the OID of
the index itself.
2018-01-17 21:56:40 +01:00
Dimitri Fontaine e010238280 Implement ALTER TABLE ... RENAME TO ...
The implementation was already mostly in place, but the code was protected
by a principled check against the operation. Turns out there's a nasty
concurrency bug though with long identifier names, much as in #1664.

To prevent deadlocks from happening, we could either review the DDL
transaction management in shards and placements, or we can simply reject
names with (NAMEDATALEN - 1) chars or more — that's because of the
PostgreSQL array types being created with a one-char prefix: '_'.
2018-01-11 13:21:24 +01:00
Marco Slot 8f69973411 Fix cancellation issues in the real-time executor (#1905) 2018-01-01 23:10:29 -05:00
Marco Slot 3fd65cb91b Do not raise errors in the real-time executor (#1903) 2018-01-01 22:26:31 -05:00
Marco Slot fa7fa2734b Log remote commands sent via MultiClientSendQuery 2017-12-22 16:18:40 +01:00
metdos 32b7e152a3 Get shard resource locks for only DMLs 2017-12-22 10:30:41 +02:00
mehmet furkan şahin fd546cf322 Intermediate result size limitation
This commit introduces a new GUC to limit the intermediate
result size which we handle when we use read_intermediate_result
function for CTEs and complex subqueries.
2017-12-21 14:26:56 +03:00
Marco Slot 7dab078e67 Set cost estimates for read_intermediate_result 2017-12-18 16:23:44 +01:00
Marco Slot 9ee0e68882 Do not take extra access exclusive lock partitioned tables 2017-12-15 13:02:31 +01:00
Marco Slot 2e2b4e81fa Add support for CTEs in distributed queries 2017-12-14 09:32:55 +01:00
Marco Slot d0335ec818 Send BEGIN for SELECTs in the router executor 2017-12-14 09:32:55 +01:00
Marco Slot 66f9f1d6cd Make some intermediate results functions public 2017-12-14 09:32:55 +01:00
Marco Slot d1a470a52e Fix issue with multiple ANALYZE in transaction block 2017-12-12 10:28:48 +01:00
mehmet furkan şahin 3c941aedf1 adds citus.enable_repartition_joins GUC
The new GUC allows Citus to switch between task executors
when necessary
2017-12-11 09:36:37 +03:00
Marco Slot 4cdadfcab6 Add intermediate results infrastructure 2017-12-04 14:50:11 +01:00
Marco Slot 73989b07eb Refactor query execution functions 2017-12-04 13:12:03 +01:00
Murat Tuncer 2d66bf5f16
Fix hard coded formatting strings for 64 bit numbers (#1831)
Postgres provides OS agnosting formatting macros for
formatting 64 bit numbers. Replaced %ld %lu with
INT64_FORMAT and UINT64_FORMAT respectively.

Also found some incorrect usages of formatting
flags and fixed them.
2017-12-04 14:11:06 +03:00
Marco Slot d6dd0b3a81 Send BEGIN in the real-time executor when in a transaction 2017-11-30 12:59:09 +01:00
Marco Slot a9933deac6 Make real time executor work in transactions 2017-11-30 09:59:32 +03:00
Jason Petersen 0eacf6bd95
Refactor VacuumStmt checker to be single-return
Decided this would be safer for the future (defaults to unsupported).
2017-11-29 16:06:50 -07:00
Jason Petersen b12e77ab0e
Ensure unsupported VACUUMs don't go to workers
Apparently these two blocks have been incorrect for nearly a year…
2017-11-29 16:06:50 -07:00
Onder Kalaci 16421f089f Register citus custom scan nodes 2017-11-23 11:38:33 +02:00
Onder Kalaci 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.
2017-11-23 11:38:12 +02:00
Marco Slot 8486f76e15 Auto-recover 2PC transactions 2017-11-22 11:26:58 +01:00
Marco Slot 6ba3f42d23 Rename MultiPlan to DistributedPlan 2017-11-22 09:36:24 +01:00
Brian Cloutier 30a2365d81 Rename CreateDirectory to CitusCreateDirectory 2017-11-20 14:38:26 -08:00
Brian Cloutier aa2ab023a2 Rename RemoveDirectory -> CitusRemoveDirectory 2017-11-20 14:21:52 -08:00
Brian Cloutier 5d9f3ae7fd Remove unused poll import from multi_real_time_executor 2017-11-14 13:09:34 -08:00
metdos 111c04c2bd Warn on CLUSTER command for distributed tables 2017-11-10 12:14:45 +02:00
Burak Yucesoy 17229ed7bd Fix attaching partition to a distributed table in schema
While attaching a partition to a distributed table in schema, we mistakenly
used unqualified name to find partitioned table's oid. This caused problems
while using partitioned tables with schemas. We are fixing this issue in
this PR.
2017-11-09 13:20:29 +03:00
Murat Tuncer 4e3d633ebf
Add check for connection failures during multishard update (#1765) 2017-11-07 12:33:25 +02:00
mehmet furkan şahin 61ae33dc7f ALTER TABLE .. REPLICA IDENTITY support is implemented 2017-10-26 13:44:28 +03:00
velioglu 0b5db5d826 Support multi shard update/delete queries 2017-10-25 15:52:38 +03:00
Marco Slot 4bde83e1d2 Relay error message if DML fails on worker 2017-10-25 14:23:21 +02:00
Marco Slot 43d5e79eaa Execute transmit commands as superuser during task-tracker queries 2017-09-28 15:27:25 +02:00
Marco Slot 306c58d59b Check for absolute paths in COPY with format transmit 2017-09-28 15:27:11 +02:00
Jason Petersen b4d53423fa
Add adapter functions for OpenFile changes 2017-09-25 17:20:24 -07:00
Jason Petersen 6c9b19a954
Add version-compat header
For polyfill macros, etc.
2017-09-25 17:20:23 -07:00
Jason Petersen 6a020b5adc
Update CopyGetAttnums with latest from PostgreSQL
This function was recently modified to use the TupleDescAttr wrapper,
which abstracts away recent changes to TupleDesc.
2017-09-25 17:20:23 -07:00
Onder Kalaci 4782f9f98a Properly copy and trim the error messages that come from pg_conn
When a NULL connection is provided to PQerrorMessage(), the
returned error message is a static text. Modifying that static
text, which doesn't necessarly be in a writeable memory, is
dangreous and might cause a segfault.
2017-09-22 19:43:09 +03:00
Jason Petersen 8b2c3fcc15
Add clarifying comment to RngVarCallbackForDropIdx
We don't need the PARTITION-related logic recently added in PostgreSQL.
2017-09-01 15:57:30 -06:00
Marco Slot 0aadbb1760 Convert multi-row INSERT target list to Vars 2017-08-25 10:55:56 +02:00
Marco Slot c97692f382 Fix multi-row INSERT with RETURNING on reference tables 2017-08-24 10:42:12 +02:00
Marco Slot 4d7927b672 Execute multi-row INSERTs sequentially 2017-08-23 10:04:57 +02:00
Marco Slot cf375d6a66 Consider dropped columns that precede the partition column in COPY 2017-08-22 13:02:35 +02:00
Onder Kalaci 6532b69873 Kill the maintenance daemon on DROP DATABASE 2017-08-18 16:03:08 +03:00