Commit Graph

21 Commits (ad791f4b3ceaa3d9fbaa679bbbfb5d6c723138df)

Author SHA1 Message Date
Naisila Puka 9d364332ac
Rename foreach_ macros to foreach_declared_ macros (#7700)
This is prep work for successful compilation with PG17

PG17added foreach_ptr, foreach_int and foreach_oid macros
Relevant PG commit
14dd0f27d7cd56ffae9ecdbe324965073d01a9ff

14dd0f27d7

We already have these macros, but they are different with the
PG17 ones because our macros take a DECLARED variable, whereas
the PG16 macros declare a locally-scoped loop variable themselves.

Hence I am renaming our macros to foreach_declared_

I am separating this into its own PR since it touches many files. The
main compilation PR is https://github.com/citusdata/citus/pull/7699
2024-10-16 17:01:39 +03:00
Jelte Fennema-Nio 48c62095ff Actually sort includes after cherry-pick 2024-04-17 10:26:50 +02:00
Marco Slot 490765a754 Enable re-partition joins after local execution 2022-02-23 19:40:21 +01:00
Marco Slot 72d8fde28b Use intermediate results for re-partition joins 2022-02-23 19:40:21 +01:00
Onur Tirtir 4dc38e9e3d
Use EnsureCompatibleLocalExecutionState instead (#5640) 2022-01-21 15:37:59 +01:00
Sait Talha Nisanci ecde6c6eef Introduce GetCurrentLocalExecutionStatus wrapper
We should not access CurrentLocalExecutionStatus directly because that
would mean that we could also set it directly, which we shouldn't
because we have checks to see if the new state is possible, otherwise we
error.
2020-10-15 15:38:19 +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
Sait Talha Nisanci 510535f558 address feedback 2020-07-13 19:45:02 +03:00
Sait Talha Nisanci db1b78148c send schema creation/cleanup to coordinator in repartitions
We were using ALL_WORKERS TargetWorkerSet while sending temporary schema
creation and cleanup. We(well mostly I) thought that ALL_WORKERS would also include coordinator when it is added as a worker. It turns out that it was FILTERING OUT the coordinator even if it is added as a worker to the cluster.

So to have some context here, in repartitions, for each jobId we create
(at least we were supposed to) a schema in each worker node in the cluster. Then we partition each shard table into some intermediate files, which is called the PARTITION step. So after this partition step each node has some intermediate files having tuples in those nodes. Then we fetch the partition files to necessary worker nodes, which is called the FETCH step. Then from the files we create intermediate tables in the temporarily created schemas, which is called a MERGE step. Then after evaluating the result, we remove the temporary schemas(one for each job ID in each node) and files.

If node 1 has file1, and node 2 has file2 after PARTITION step, it is
enough to either move file1 from node1 to node2 or vice versa. So we
prune one of them.

In the MERGE step, if the schema for a given jobID doesn't exist, the
node tries to use the `public` schema if it is a superuser, which is
actually added for testing in the past.

So when we were not sending schema creation comands for each job ID to
the coordinator(because we were using ALL_WORKERS flag, and it doesn't
include the coordinator), we would basically not have any schemas for
repartitions in the coordinator. The PARTITION step would be executed on
the coordinator (because the tasks are generated in the planner part)
and it wouldn't give us any error because it doesn't have anything to do
with the temporary schemas(that we didn't create). But later two things
would happen:

- If by chance the fetch is pruned on the coordinator side, we the other
nodes would fetch the partitioned files from the coordinator and execute
the query as expected, because it has all the information.
- If the fetch tasks are not pruned in the coordinator, in the MERGE
step, the coordinator would either error out saying that the necessary
schema doesn't exist, or it would try to create the temporary tables
under public schema ( if it is a superuser). But then if we had the same
task ID with different jobID it would fail saying that the table already
exists, which is an error we were getting.

In the first case, the query would work okay, but it would still not do
the cleanup, hence we would leave the partitioned files from the
PARTITION step there. Hence ensure_no_intermediate_data_leak would fail.

To make things more explicit and prevent such bugs in the future,
ALL_WORKERS is named as ALL_NON_COORD_WORKERS. And a new flag to return
all the active nodes is added as ALL_DATA_NODES. For repartition case,
we don't use the only-reference table nodes but this version makes the
code simpler and there shouldn't be any significant performance issue
with that.
2020-07-13 19:20:15 +03:00
SaitTalhaNisanci 3f50165365
rename TargetWorkerSet enums (#4015)
Rename TargetWorkerSet enums to make them more explicit about what they
mean. Ideally it would be good to treat everything as a node without the
'worker' concept because it makes things complicated. Another
improvement could be to rename TargetWorkerSet as TargetNodeSet but it
goes to renaming many occurrences of Worker, which is probably too big
for this PR.
2020-07-10 11:21:27 +03:00
SaitTalhaNisanci f458d1fd1c
Fix/task execution (#3941)
* Not set TaskExecution with adaptive executor

Adaptive executor is using a utility method from task tracker for
repartition joins, however adaptive executor doesn't need taskExecution.
It is only used by task tracker. This causes a problem when explain
analyze is used because what taskExecution is pointing to might be
random.

We solve this by not setting taskExecution from adaptive executor. So it
will stay NULL as set by CreateTask.

* use same memory context as task for taskExecution

Co-authored-by: Jelte Fennema <github-tech@jeltef.nl>
2020-06-24 12:10:00 +03:00
SaitTalhaNisanci 3dc7cad754
use an enum for local execution status (#3733)
We have two variables that are related to local execution status.
TransactionAccessedLocalPlacement and
TransactionConnectedToLocalGroup. Only one of these fields should be
set, however we didn't have any check for this contraint and it was
error prone.

What those two variables are used is that we are trying to understand if
we should use local execution, the current session, or if we should be
using a connection to execute the current query, therefore the tasks. In
the enum, now it is more clear what these variables mean.

Also, now we have a method to change the local execution status. The
method will error if we are trying to transition from a state to a wrong
state. This will help us avoid problems.
2020-04-09 19:11:04 +03:00
SaitTalhaNisanci fa88046ce1
test that we don't leak intermediate schemas (#3737)
* test that we don't leak intermediate schemas

We have tests to make sure that we don't intermediate any intermediate
files, tables etc but we don't test if we are leaking schemas. It makes
sense to test this as well.

* remove all repartition schemas in case of error

This solution is not an ideal one but it seems to be doing the job.
We should have a more generic solution for the cleanup but it seems that
putting the cleanup in the abort handler is dangerous and it was
crashing.
2020-04-09 12:17:41 +03:00
Philip Dubé 2fae132e45
repartition_join_execution: Don't store 64 bit integers as poin… (#3551)
Pointers are not necessarily 64bit
2020-02-28 15:06:06 +01:00
Philip Dubé 20abc4d2b5
Replace foreach with foreach_ptr/foreach_oid (#3544) 2020-02-27 16:54:49 +01:00
SaitTalhaNisanci 82d22b34fe
create temp schemas in parallel (#3540) 2020-02-26 16:20:08 +03:00
SaitTalhaNisanci d94c3fd43d
send repartition cleanup jobs in parallel to all workers (#3485)
* send repartition cleanup jobs in parallel to all workers

* add review items
2020-02-26 13:44:06 +03:00
Philip Dubé ecad4aa5e6 Fill in jobIdList field of DistributedExecution
Pass down jobIdList from ExecuteTasksInDependencyOrder

Also clean up comment for ExecuteTaskListOutsideTransaction
2020-02-05 17:32:22 +00:00
Philip Dubé c252811884 dont: don't, wont: won't, acylic: acyclic 2020-02-05 17:32:22 +00:00
Hadi Moshayedi bc1a800f70 Use current user for repartition join temp schemas.
Otherwise when using a less privileged user we might get
errors when trying to create the schema.
2020-02-04 09:48:20 -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