Commit Graph

151 Commits (9d364332ac1b30920847e39ab55a7f6fa6941ff9)

Author SHA1 Message Date
Onur Tirtir 3a73fba810 Apply planner changes for citus local tables 2020-09-09 11:51:18 +03:00
SaitTalhaNisanci 366461ccdb
Introduce cache entry/table utilities (#4132)
Introduce table entry utility functions

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

* Introduce utility functions to find the type of tables

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

* Add IsCitusTableType utilities

* Rename IsCacheEntryCitusTableType -> IsCitusTableTypeCacheEntry

* Change citus table types in some checks
2020-09-02 22:26:05 +03:00
Sait Talha Nisanci d68bfc5687 Improve error for index operator class parameters
The error message when index has opclassopts is improved and the commit
from postgres side is also included for future reference.

Also some minor style related changes are applied.
2020-08-04 15:38:13 +03:00
Sait Talha Nisanci 1112b254a7 adapt recently added code for pg13
This commit mostly adds pg_get_triggerdef_command to our ruleutils_13.
This doesn't add anything extra for ruleutils 13 so it is basically a copy
of the change on ruleutils_12
2020-08-04 15:18:27 +03:00
Sait Talha Nisanci 9f1ec792b3 add queryString to distributed_planner
distributed_planner now takes query string as a parameter.

related commit on PG side:
6aba63ef3e606db71beb596210dd95fa73c44ce2
2020-08-04 15:10:22 +03:00
Sait Talha Nisanci 6314eba5df introduce standard_planner_compat
standard_planner now takes the query string as a parameter as well with
pg >= 13.

Commit on Postgres Side:
66888f7424f7d6c7cea2c26e181054d1455d4e7a
2020-08-04 15:10:22 +03:00
Sait Talha Nisanci 4308d867d9 remove task-tracker in comments, documentation 2020-07-21 16:21:01 +03:00
SaitTalhaNisanci b3af63c8ce
Remove task tracker executor (#3850)
* use adaptive executor even if task-tracker is set

* Update check-multi-mx tests for adaptive executor

Basically repartition joins are enabled where necessary. For parallel
tests max adaptive executor pool size is decresed to 2, otherwise we
would get too many clients error.

* Update limit_intermediate_size test

It seems that when we use adaptive executor instead of task tracker, we
exceed the intermediate result size less in the test. Therefore updated
the tests accordingly.

* Update multi_router_planner

It seems that there is one problem with multi_router_planner when we use
adaptive executor, we should fix the following error:
+ERROR:  relation "authors_range_840010" does not exist
+CONTEXT:  while executing command on localhost:57637

* update repartition join tests for check-multi

* update isolation tests for repartitioning

* Error out if shard_replication_factor > 1 with repartitioning

As we are removing the task tracker, we cannot switch to it if
shard_replication_factor > 1. In that case, we simply error out.

* Remove MULTI_EXECUTOR_TASK_TRACKER

* Remove multi_task_tracker_executor

Some utility methods are moved to task_execution_utils.c.

* Remove task tracker protocol methods

* Remove task_tracker.c methods

* remove unused methods from multi_server_executor

* fix style

* remove task tracker specific tests from worker_schedule

* comment out task tracker udf calls in tests

We were using task tracker udfs to test permissions in
multi_multiuser.sql. We should find some other way to test them, then we
should remove the commented out task tracker calls.

* remove task tracker test from follower schedule

* remove task tracker tests from multi mx schedule

* Remove task-tracker specific functions from worker functions

* remove multi task tracker extra schedule

* Remove unused methods from multi physical planner

* remove task_executor_type related things in tests

* remove LoadTuplesIntoTupleStore

* Do initial cleanup for repartition leftovers

During startup, task tracker would call TrackerCleanupJobDirectories and
TrackerCleanupJobSchemas to clean up leftover directories and job
schemas. With adaptive executor, while doing repartitions it is possible
to leak these things as well. We don't retry cleanups, so it is possible
to have leftover in case of errors.

TrackerCleanupJobDirectories is renamed as
RepartitionCleanupJobDirectories since it is repartition specific now,
however TrackerCleanupJobSchemas cannot be used currently because it is
task tracker specific. The thing is that this function is a no-op
currently.

We should add cleaning up intermediate schemas to DoInitialCleanup
method when that problem is solved(We might want to solve it in this PR
as well)

* Revert "remove task tracker tests from multi mx schedule"

This reverts commit 03ecc0a681.

* update multi mx repartition parallel tests

* not error with task_tracker_conninfo_cache_invalidate

* not run 4 repartition queries in parallel

It seems that when we run 4 repartition queries in parallel we get too
many clients error on CI even though we don't get it locally. Our guess
is that, it is because we open/close many connections without doing some
work and postgres has some delay to close the connections. Hence even
though connections are removed from the pg_stat_activity, they might
still not be closed. If the above assumption is correct, it is unlikely
for it to happen in practice because:
- There is some network latency in clusters, so this leaves some times
for connections to be able to close
- Repartition joins return some data and that also leaves some time for
connections to be fully closed.

As we don't get this error in our local, we currently assume that it is
not a bug. Ideally this wouldn't happen when we get rid of the
task-tracker repartition methods because they don't do any pruning and
might be opening more connections than necessary.

If this still gives us "too many clients" error, we can try to increase
the max_connections in our test suite(which is 100 by default).

Also there are different places where this error is given in postgres,
but adding some backtrace it seems that we get this from
ProcessStartupPacket. The backtraces can be found in this link:
https://circleci.com/gh/citusdata/citus/138702

* Set distributePlan->relationIdList when it is needed

It seems that we were setting the distributedPlan->relationIdList after
JobExecutorType is called, which would choose task-tracker if
replication factor > 1 and there is a repartition query. However, it
uses relationIdList to decide if the query has a repartition query, and
since it was not set yet, it would always think it is not a repartition
query and would choose adaptive executor when it should choose
task-tracker.

* use adaptive executor even with shard_replication_factor > 1

It seems that we were already using adaptive executor when
replication_factor > 1. So this commit removes the check.

* remove multi_resowner.c and deprecate some settings

* remove TaskExecution related leftovers

* change deprecated API error message

* not recursively plan single relatition repartition subquery

* recursively plan single relation repartition subquery

* test depreceated task tracker functions

* fix overlapping shard intervals in range-distributed test

* fix error message for citus_metadata_container

* drop task-tracker deprecated functions

* put the implemantation back to worker_cleanup_job_schema_cachesince citus cloud uses it

* drop some functions, add downgrade script

Some deprecated functions are dropped.
Downgrade script is added.
Some gucs are deprecated.
A new guc for repartition joins bucket size is added.

* order by a test to fix flappiness
2020-07-18 13:11:36 +03:00
Marco Slot b4fec63bc0 Rename master evaluation to coordinator evaluation 2020-07-07 10:37:41 +02:00
Hadi Moshayedi d34c21890f Rename CoordinatorInsertSelect... to NonPushableInsertSelect 2020-06-25 08:55:48 -07:00
Hadi Moshayedi 4e8d79998e Save INSERT/SELECT method in DistributedPlan.
This is so we don't need to calculate it twice in
insert_select_executor.c and multi_explain.c, which can
cause discrepancy if an update in one of them is not
reflected in the other site.
2020-06-25 08:55:48 -07:00
Marco Slot 2a3234ca26 Rename masterQuery to combineQuery 2020-06-17 14:14:37 +02:00
Marco Slot d1bab78d79 Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
Marco Slot 24feadc230 Handle joins between local/reference/cte via router planner 2020-06-12 18:36:01 -07:00
Halil Ozan Akgül 8c5eb6b7ea
Insert Select Into Local Table (#3870)
* Insert select with master query

* Use relid to set custom_scan_tlist varno

* Reviews

* Fixes null check

Co-authored-by: Marco Slot <marco.slot@gmail.com>
2020-06-12 17:06:31 +03:00
Onur Tirtir a4f1c41391
Implement GetQueryLockMode helper (#3860)
If we want to get necessary lockmode for a relation RangeVar within
a query, we can get the lockmode easily from the RangeVar itself (if
pg version >= 12).

However, if we want to decide the lockmode appropriate for the
"query", we can derive this information by using GetQueryLockMode
according to the code comment from RangeTblEntry->rellockmode.
2020-06-09 13:08:44 +03:00
Philip Dubé c0515dcd67 This prepares for routing modifying CTEs, where modLevel should not be used to infer whether a plan is a select or not
SELECT_TASK is renamed to READ_TASK as a SELECT with modifying CTEs will be a MODIFYING_TASK

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

Remove unused function IsModifyDistributedPlan

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

Rename CreateSingleTaskRouterPlan to CreateSingleTaskRouterSelectPlan
2020-05-20 17:26:12 +00:00
Onder Kalaci 0cb7ab2d05 Explicitly mark queries in physical planner for [not] having parameters
Physical planner doesn't support parameters. If the parameters have already
been resolved when the physical planner handling the queries, mark it.
The reason is that the executor is unaware of this, and sends the parameters
along with the worker queries, which fails for composite types.

(See `DissuadePlannerFromUsingPlan()` for the details of paramater resolving)
2020-04-24 12:49:43 +02:00
Marco Slot 8b83306a27 Issue worker messages with the same log level 2020-04-14 21:08:25 +02:00
Marco Slot 84672c3dbd Simplify intermediate result pruning logic 2020-04-07 10:53:29 +02:00
Onur Tirtir 4c95ad1579 do not traverse parse tree in distributed planner one more time 2020-04-03 18:24:48 +03:00
Onur Tirtir abdabbedb2 refactor distributed_planner.c 2020-04-03 18:24:41 +03:00
SaitTalhaNisanci ba01f3457a
use macros for pg versions instead of hardcoded values (#3694)
3 Macros are defined for removing the hardcoded pg versions.
PG_VERSION_11, PG_VERSION_12 and PG_VERSION_13.
2020-04-01 17:01:52 +03:00
Philip Dubé 7cdfa1daab Rename LookupCitusTableCacheEntry to GetCitusTableCacheEntry, LookupLookupCitusTableCacheEntry back to LookupCitusTableCacheEntry 2020-03-08 14:08:23 +00:00
Philip Dubé a7cca1bcde Rename DistTableCacheEntry to CitusTableCacheEntry 2020-03-07 14:08:03 +00:00
Philip Dubé bec58000d6 Given IsDistributedTableRTE, there's ambiguity in what DistributedTable means
Elsewhere we used DistributedTable to include reference tables
Marco suggested we use CitusTable for distributed & reference tables

So renaming:
- IsDistributedTable -> IsCitusTable
- IsDistributedTableViaCatalog -> IsCitusTableViaCatalog
- DistributedTableCacheEntry -> CitusTableCacheEntry
- DistributedTableList -> CitusTableList
- isDistributedTable -> isCitusTable
- InsertSelectIntoDistributedTable -> InsertSelectIntoCitusTable
- ExtractFirstDistributedTableId -> ExtractFirstCitusTableId
2020-03-06 18:57:55 +00:00
Jelte Fennema 685b54b3de
Semmle: Check for NULL in some places where it might occur (#3509)
Semmle reported quite some places where we use a value that could be NULL. Most of these are not actually a real issue, but better to be on the safe side with these things and make the static analysis happy.
2020-02-27 10:45:29 +01:00
Jelte Fennema 8de8b62669 Convert unsafe APIs to safe ones 2020-02-25 15:39:27 +01:00
Nils Dijk a77ed9cd23
Refactor master query to be planned by postgres' planner (#3326)
DESCRIPTION: Replace the query planner for the coordinator part with the postgres planner

Closes #2761 

Citus had a simple rule based planner for the query executed on the query coordinator. This planner grew over time with the addigion of SQL support till it was getting close to the functionality of the postgres planner. Except the code was brittle and its complexity rose which made it hard to add new SQL support.

Given its resemblance with the postgres planner it was a long outstanding wish to replace our hand crafted planner with the well supported postgres planner. This patch replaces our planner with a call to postgres' planner.

Due to the functionality of the postgres planner we needed to support both projections and filters/quals on the citus custom scan node. When a sort operation is planned above the custom scan it might require fields to be reordered in the custom scan before returning the tuple (projection). The postgres planner assumes every custom scan node implements projections. Because we controlled the plan that was created we prevented reordering in the custom scan and never had implemented it before.

A same optimisation applies to having clauses that could have been where clauses. Instead of applying the filter as a having on the aggregate it will push it down into the plan which could reach a custom scan node.

For both filters and projections we have implemented them when tuples are read from the tuple store. If no projections or filters are required it will directly return the tuple from the tuple store. Otherwise it will loop tuples from the tuple store through the filter and projection until a tuple is found and returned.

Besides filters being pushed down a side effect of having quals that could have been a where clause is that a call to read intermediate result could be called before the first tuple is fetched from the custom scan. This failed because the intermediate result would only be pulled to the coordinator on the first tuple fetch. To overcome this problem we do run the distributed subplans now before we run the postgres executor. This ensures the intermediate result is present on the coordinator in time. We do account for total time instrumentation by removing the instrumentation before handing control to the psotgres executor and update the timings our self.

For future SQL support it is enough to create a valid query structure for the part of the query to be executed on the query coordinating node. As a utility we do serialise and print the query at debug level4 for engineers to inspect what kind of query is being planned on the query coordinator.
2020-02-25 14:39:56 +01:00
Jelte Fennema 2a9fccc7a0
Remove READFUNCs (#3536)
We don't actually use these functions anymore since merging #1477.

Advantages of removing:
1. They add work whenever we add a new node.
2. They contain some usage of stdlib APIs that are banned by Microsoft.
   Removing it means we don't have to replace those with safe ones.
2020-02-24 12:43:28 +01:00
Philip Dubé 08f6842d50 Fix typos
Equivalance -> Equivalence
utillity -> utility
shorted lived one -> shortly lived one
elegible -> eligible
2020-02-18 17:14:40 +00:00
Philip Dubé 3a906b8210 Fix typos noticed while reading through code trying to understand HAVING 2020-02-11 19:55:10 +00:00
Önder Kalacı 4519d3411d
Improve the representation of used sub plans (#3411)
Previously, we've identified the usedSubPlans by only looking
to the subPlanId.

With this commit, we're expanding it to also include information
on the location of the subPlan.

This is useful to distinguish the cases where the subPlan is used
either on only HAVING or both HAVING and any other part of the query.
2020-01-24 10:47:14 +01:00
Onder Kalaci 3833a7e686 Fix issues for CTE inlining on Postgres 11
Comment from code:

/*
 * We had to implement this hack because on Postgres11 and below, the originalQuery
 * and the query would have significant differences in terms of CTEs where CTEs
 * would not be inlined on the query (as standard_planner() wouldn't inline CTEs
 * on PG 11 and below).
 *
 * Instead, we prefer to pass the inlined query to the distributed planning. We rely
 * on the fact that the query includes subqueries, and it'd definitely go through
 * query pushdown planning. During query pushdown planning, the only relevant query
 * tree is the original query.
 */
2020-01-17 11:59:02 +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
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
Onder Kalaci 64560b07be Update regression tests-2
In this commit, we're introducing a way to prevent CTE inlining via a GUC.

The GUC is used in all the tests where PG 11 and PG 12 tests would diverge
otherwise.

Note that, in PG 12, the restriction information for CTEs are generated. It
means that for some queries involving CTEs, Citus planner (router planner/
pushdown planner) may behave differently. So, via the GUC, we prevent
tests to diverge on PG 11 vs PG 12.

When we drop PG 11 support, we should get rid of the GUC, and mark
relevant ctes as MATERIALIZED, which does the same thing.
2020-01-16 12:28:15 +01:00
Onder Kalaci 05d600dd8f Call CTE inlining in Citus planner
The idea is simple: Inline CTEs(if any), try distributed planning.
If the planning yields a successful distributed plan, simply return
it.

If the planning fails, fallback to distributed planning on the query
tree where CTEs are not inlined. In that case, if the planning failed
just because of the CTE inlining, via recursive planning, the same
query would yield a successful plan.

A very basic set of examples:

WITH cte_1 AS (SELECT * FROM test_table)
SELECT
	*, row_number() OVER ()
FROM
	cte_1;

or

WITH a AS (SELECT * FROM test_table),
b AS (SELECT * FROM test_table)
SELECT * FROM  a JOIN b ON (a.value> b.value);
2020-01-16 12:28:15 +01:00
Jelte Fennema 5b0baea72c Refactor distributed_planner for better understandability 2020-01-06 14:23:38 +01:00
Onder Kalaci 5a1e752726 Apply feedback - add fastPath field to plan 2020-01-06 12:42:43 +01:00
Onder Kalaci 13a9b55695 Skip expensive checks when fast-path query
The definition of fast-path query is very strict. So, we don't need
to do some extra checks.
2020-01-06 12:42:43 +01:00
Onder Kalaci 7f3ab7892d Skip shard pruning when possible
We're already traversing the queryTree and finding the distribution
key value, so pass it to the later stages of the planning.
2020-01-06 12:42:43 +01:00
Onder Kalaci ca293116fa Reduce calls to FastPathRouterQuery()
Before this commit, we called it twice durning planning. Instead,
we save the information and pass it.
2020-01-06 12:42:43 +01:00
Hadi Moshayedi 249508d267 Estimate cost of read_intermediate_results() 2019-12-17 13:51:51 -08:00
Marco Slot 5f656e22db Fix issue in IsMultiStatementTransaction detection 2019-12-16 17:01:43 +01:00
Hadi Moshayedi e7a6cc0801 Fix some typos from #3280 2019-12-12 13:29:26 -08:00
Hadi Moshayedi 939d3c955b Don't plan function joins locally 2019-12-11 16:53:29 -08:00
Hadi Moshayedi 067d92a7f6 Don't plan joins between ref tables and views locally 2019-12-11 14:31:34 -08:00
Philip Dubé 0d04ff1692 RECORD: Add support for more expression types
- OpExpr
- NullIfExpr
- MinMaxExpr
- CoalesceExpr
- CaseExpr

Also fix case where ARRAY[(1,2), NULL] was rejected
2019-11-27 17:07:22 +00:00
Philip Dubé 168e11cc9b Implement support for RECORD[] where we support RECORD
Support for ARRAY[] expressions is limited to having a consistent shape,
eg ARRAY[(int,text),(int,text)] as opposed to ARRAY[(int,text),(float,text)] or ARRAY[(int,text),(int,text,float)]
2019-11-27 15:02:43 +00: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
Philip Dubé 99164398bf Fix potential segfault from standard_planner inlining functions 2019-11-21 18:47:36 +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
Jelte Fennema 1b2c438e69
Rename variables to not shadow globals in RHEL6 (#3194)
Fixes #2839
2019-11-15 12:12:24 +01:00
Philip Dubé edc7a2ee38 Improve RECORD support 2019-11-14 18:32:22 +00:00
Jelte Fennema adc6ca6100
Make simple in queries on unique columns work with repartion join (#3171)
This is necassery to support Q20 of the CHbenCHmark: #2582.

To summarize the fix: The subquery is converted into an INNER JOIN on a
table. This fixes the issue, since an INNER JOIN on a table is already
supported by the repartion planner.

The way this replacement is happening.:
1. Postgres replaces `col in (subquery)` with a SEMI JOIN (subquery) on col = subquery_result
2. If this subquery is simple enough Postgres will replace it with a
   regular read from a table
3. If the subquery returns unique results (e.g. a primary key) Postgres
   will convert the SEMI JOIN into an INNER JOIN during the planning. It
   will not change this in the rewritten query though.
4. We check if Postgres sends us any SEMI JOINs during its join order
   planning, if it doesn't we replace all SEMI JOINs in the rewritten
   query with INNER JOIN (which we already support).
2019-11-11 13:44:28 +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
Ö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
Onur TIRTIR d5f83dc110
Refactor range table walkers (#3109) 2019-10-16 01:20:49 +03:00
SaitTalhaNisanci 94a7e6475c
Remove copyright years (#2918)
* Update year as 2012-2019

* Remove copyright years
2019-10-15 17:44:30 +03:00
Philip Dubé 74cb168205 Remove Postgres 10 support 2019-10-11 21:56:56 +00:00
Philip Dubé 89d35e9692 Attempt to force custom plans for prepared statements when trying to delegate function calls
We discern between PARAM_EXEC & PARAM_EXTERN:
d52eaa0948/src/include/nodes/primnodes.h (L211)
According to primnodes.h we should only run into PARAM_EXEC or PARAM_EXTERN
2019-09-30 23:49:14 +00: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
Philip Dubé 06faba91c0 Include ifdefs for pg12 API changes, update local_shard_executiuon test to avoid CTE inlining 2019-09-23 20:22:35 +00:00
Marco Slot d85d77634d Handle anonymous composite types on the target list 2019-09-23 14:53:02 +02:00
Philip Dubé bee779e7d4 planner/distributed_planner.c: get_func_cost replaced with add_function_cost in pg12 2019-08-22 19:02:10 +00:00
Philip Dubé 0915027389 DistributedPlan: replace operation with modLevel
This causes no behaviorial changes, only organizes better to implement modifying CTEs

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

Remove Task's upsertQuery in favor of ROW_MODIFY_NONCOMMUTATIVE

Split up AcquireExecutorShardLock into more internal functions

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


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

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



All the commits involved here:
* Initial unified executor prototype

* Latest changes

* Fix rebase conflicts to master branch

* Add missing variable for assertion

* Ensure that master_modify_multiple_shards() returns the affectedTupleCount

* Adjust intermediate result sizes

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

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

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

That might be useful under certain circumstances:

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

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

* For parameters to be resolved before using them

For the details, see PostgreSQL's copyParamList()

* Unified executor sorts the returning output

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

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

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

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

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

* Fix 2PC conditions for DDL tasks

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

* Support foreign keys to reference tables in unified executor

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

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

* Make sure to close the failed connections after the execution

* Improve comments

* Fix savepoints in unified executor.

* Rebuild the WaitEventSet only when necessary

* Unclaim connections on all errors.

* Improve failure handling for unified executor

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

Improve the logic to start a transaction block for distributed transactions

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

* Move waitEventSet to DistributedExecution

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

* Fix multi_insert_select test outputs

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

* Fix subeury_and_cte test

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

which is a lot clearer to the user.

* Fix intermediate_results test outputs

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

which makes a lot more sense.

* Fix multi_function_in_join test

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

* Fix multi_query_directory_cleanup test

The unified executor does not create any intermediate files.

* Fix with_transactions test

A test case that just started to work fine

* Fix multi_router_planner test outputs

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

which is a lot more clearer for the users

* Fix multi_router_planner_fast_path test

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

which is a lot more clearer for the users

* Fix isolation_copy_placement_vs_modification by disabling select_opens_transaction_block

* Fix ordering in isolation_multi_shard_modify_vs_all

* Add executor locks to unified executor

* Make sure to allocate enought WaitEvents

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

* Fix rebase conflicts for master rebase

* Make sure that TRUNCATE relies on unified executor

* Implement true sequential execution for multi-row INSERTS

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

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

* Remove SESSION_LIFESPAN flag in unified_executor

* Apply failure test updates

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

* Unified executor honors citus.node_connection_timeout

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

And, as a side effect this fixes failure_connection_establishment
test.

* Properly increment/decrement pool size variables

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

* insert_select_executor goes through unified executor.

* Add missing file for task tracker

* Modify ExecuteTaskListExtended()'s signature

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

* Take partition locks correctly in unified executor

* Alternative implementation for force_max_query_parallelization

* Fix compile warnings in unified executor

* Fix style issues

* Decrement idleConnectionCount when idle connection is lost

* Always rebuild the wait event sets

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

* Make sure to allocate enough sized waitEventSet

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

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

* Implement SELECT FOR UPDATE on reference tables

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

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

* SELECT FOR UPDATE opens transaction block even if SelectOpensTransactionBlock disabled

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

* Fix build error

* Fix the newConnectionCount calculation in ManageWorkerPool

* Fix rebase conflicts

* Fix minor test output differences

* Fix citus indent

* Remove duplicate sorts that is added with rebase

* Create distributed table via executor

* Fix wait flags in CheckConnectionReady

* failure_savepoints output for unified executor.

* failure_vacuum output (pg 10) for unified executor.

* Fix WaitEventSetWait timeout in unified executor

* Stabilize failure_truncate test output

* Add an ORDER BY to multi_upsert

* Fix regression test outputs after rebase to master

* Add executor.c comment

* Rename executor.c to adaptive_executor.c

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

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

* Implement a proper custom scan node for adaptive executor

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

* Add slow start interval to the executor

* Expose max_cached_connection_per_worker to user

* Do not start slow when there are cached connections

* Consider ExecutorSlowStartInterval in NextEventTimeout

* Fix memory issues with ReceiveResults().

* Disable executor via TaskExecutorType

* Make sure to execute the tests with the other executor

* Use task_executor_type to enable-disable adaptive executor

* Remove useless code

* Adjust the regression tests

* Add slow start regression test

* Rebase to master

* Fix test failures in adaptive executor.

* Rebase to master - 2

* Improve comments & debug messages

* Set force_max_query_parallelization in isolation_citus_dist_activity

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

* Adjust the default pool size

* Expand description of max_adaptive_executor_pool_size GUC

* Update warnings in FinishRemoteTransactionCommit()

* Improve session clean up at the end of execution

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

* Remove MULTI_CONNECTION_WAIT_RETRY which is not used at all

* Add more ORDER BYs to multi_mx_partitioning
2019-06-28 14:04:40 +02:00
Philip Dubé 77efec04a0 Router Planner: accept SELECT_CMD ctes in modification queries 2019-06-26 10:32:01 +02:00
Philip Dubé 84fe626378 multi_router_planner: refactor error propagation 2019-06-26 10:32:01 +02:00
Hadi Moshayedi 8e2d328530 Search all outer node levels for lateral join params. 2019-06-04 10:14:05 -07:00
Marco Slot ee6a0b6943 Speed up RTE walkers
Do it in two ways (a) re-use the rte list as much as possible instead of
re-calculating over and over again (b) Limit the recursion to the relevant
parts of the query tree
2019-03-20 12:14:46 +03:00
Onder Kalaci f144bb4911 Introduce fast path router planning
In this context, we define "Fast Path Planning for SELECT" as trivial
queries where Citus can skip relying on the standard_planner() and
handle all the planning.

For router planner, standard_planner() is mostly important to generate
the necessary restriction information. Later, the restriction information
generated by the standard_planner is used to decide whether all the shards
that a distributed query touches reside on a single worker node. However,
standard_planner() does a lot of extra things such as cost estimation and
execution path generations which are completely unnecessary in the context
of distributed planning.

There are certain types of queries where Citus could skip relying on
standard_planner() to generate the restriction information. For queries
in the following format, Citus does not need any information that the
standard_planner() generates:

  SELECT ... FROM single_table WHERE distribution_key = X;  or
  DELETE FROM single_table WHERE distribution_key = X; or
  UPDATE single_table SET value_1 = value_2 + 1 WHERE distribution_key = X;

Note that the queries might not be as simple as the above such that
GROUP BY, WINDOW FUNCIONS, ORDER BY or HAVING etc. are all acceptable. The
only rule is that the query is on a single distributed (or reference) table
and there is a "distribution_key = X;" in the WHERE clause. With that, we
could use to decide the shard that a distributed query touches reside on
a worker node.
2019-02-21 13:27:01 +03:00
velioglu 1bb0ec316a Reset planner restriction context instead of popping with recursive planning 2019-01-17 14:35:16 +03: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 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
Onder Kalaci 9e2e2a7300 Make sure to access PARAM_EXTERN accurately in PG 11
PG 11 has change the way that PARAM_EXTERN is processed.
This commit ensures that Citus follows the same pattern.

For details see the related Postgres commit:
6719b238e8
2018-10-25 21:55:03 +03:00
Jason Petersen ae9a98c2d1
Attempt to address planner context crashes
Both of these are a bit of a shot in the dark. In one case, we noticed
a stack trace where a caller received a null pointer and attempted to
dereference the memory context field (at 0x010). In the other, I saw
that any error thrown from within AdjustParseTree could keep the stack
from being cleaned up (presumably if we push we should always pop).

Both stack traces were collected during times of high memory pressure
and locally reproducing the problem locally or otherwise has been very
tricky (i.e. it hasn't been reproduced reliably at all).
2018-10-18 08:41:51 -06:00
Marco Slot 1ca9a5b867 Do not allow unresolved parameters in INSERT...SELECT 2018-09-24 14:12:04 +02: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
Onder Kalaci 41d606b575 Use tree walker instad of mutator in relation visibility
This commit uses *_walker instead of *_mutator for performance reasons.
Given that we're only updating a functionId in the tree, the approach
seems fine.
2018-09-18 09:33:01 +03:00
Onder Kalaci 974cbf11a5 Hide shard names on MX worker nodes
This commit by default enables hiding shard names on MX workers
by simple replacing `pg_table_is_visible()` calls with
`citus_table_is_visible()` calls on the MX worker nodes. The latter
function filters out tables that are known to be shards.

The main motivation of this change is a better UX. The functionality
can be opted out via a GUC.

We also added two views, namely citus_shards_on_worker and
citus_shard_indexes_on_worker such that users can query
them to see the shards and their corresponding indexes.

We also added debug messages such that the filtered tables can
be interactively seen by setting the level to DEBUG1.
2018-08-07 14:21:45 +03:00
Murat Tuncer 4d35b92016 Add groundwork for citus_stat_statements api 2018-06-27 14:20:03 +03:00
Marco Slot 2f9c8c6af0 Allow DML commands with unreferenced SELECT CTEs 2018-05-03 14:53:26 +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
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
Marco Slot ee6a751798 Only copy distributed plan when modifying it 2018-02-12 16:30:55 +01:00
Onder Kalaci 94c5ac6ebb Remove duplicate join restrictions
We use PostgreSQL hooks to accumulate the join restrictions
and PostgreSQL gives us all the join paths it tries while
deciding on the join order. Thus, for queries that have many
joins, this function is likely to remove lots of duplicate join
restrictions. This becomes relevant for Citus on query pushdown
check peformance.
2018-02-12 18:35:05 +02:00
Onder Kalaci 0d5a4b9c72 Recursively plan subqueries that are not safe to pushdown
With this commit, Citus recursively plans subqueries that
are not safe to pushdown, in other words, requires a merge
step.

The algorithm is simple: Recursively traverse the query from bottom
up (i.e., bottom meaning the leaf queries). On each level, check
whether the query is safe to pushdown (or a single repartition
subquery). If the answer is yes, do not touch that subquery. If the
answer is no, plan the subquery seperately (i.e., create a subPlan
for it) and replace the subquery with a call to
`read_intermediate_results(planId, subPlanId)`. During the the
execution, run the subPlans first, and make them avaliable to the
next query executions.

Some of the queries hat this change allows us:

   * Subqueries with LIMIT
   * Subqueries with GROUP BY/DISTINCT on non-partition keys
   * Subqueries involving re-partition joins, router queries
   * Mixed usage of subqueries and CTEs (i.e., use CTEs in
     subqueries as well). Nested subqueries as long as we
     support the subquery inside the nested subquery.
   * Subqueries with local tables (i.e., those subqueries
     has the limitation that they have to be leaf subqueries)

   * VIEWs on the distributed tables just works (i.e., the
     limitations mentioned below still applies to views)

Some of the queries that is still NOT supported:

  * Corrolated subqueries that are not safe to pushdown
  * Window function on non-partition keys
  * Recursively planned subqueries or CTEs on the outer
    side of an outer join
  * Only recursively planned subqueries and CTEs in the FROM
    (i.e., not any distributed tables in the FROM) and subqueries
    in WHERE clause
  * Subquery joins that are not on the partition columns (i.e., each
    subquery is individually joined on partition keys but not the upper
    level subquery.)
  * Any limitation that logical planner applies such as aggregate
    distincts (except for count) when GROUP BY is on non-partition key,
    or array_agg with ORDER BY
2017-12-21 08:37:40 +02:00
Onder Kalaci 71ce42b936 Refactor RecursivelyPlanSubqueriesAndCTEs() to make it ready
to work with subqueries
2017-12-20 09:03:47 +02:00
Marco Slot 7dab078e67 Set cost estimates for read_intermediate_result 2017-12-18 16:23:44 +01:00
Marco Slot 2e2b4e81fa Add support for CTEs in distributed queries 2017-12-14 09:32:55 +01:00
Marco Slot 60a1e31671 Allow queries with local tables in NeedsDistributedPlanning 2017-12-07 16:20:23 +01:00
Marco Slot d8fea4efb8 Revert "Allow queries with local tables in NeedsDistributedPlanning"
This reverts commit d2bac081e8.
2017-12-07 11:19:11 +01:00
Marco Slot d2bac081e8 Allow queries with local tables in NeedsDistributedPlanning 2017-12-07 11:02:16 +01:00
Onder Kalaci 32def06ebd Split assigning RTE identities and partitioning related query modifications
Note that we used to iterate over the RTEs once for performance reasons.
However, keeping an extra copy of original query seems more costly and
hard to maintain/explain.
2017-11-28 09:27:34 +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