citus/src/include/distributed
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
..
commands Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
metadata move MakeNameListFromRangeVar function to a more appropriate file 2020-03-25 11:01:50 +03:00
adaptive_executor.h Support use of binary protocol in between nodes (#3877) 2020-06-12 15:02:51 +02:00
argutils.h Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
backend_data.h Fix wrong cancellation message about distributed deadlocks (#3956) 2020-06-30 14:57:46 +02:00
cancel_utils.h add IsHoldOffCancellationReceived utility function (#3290) 2019-12-12 17:32:59 +03:00
citus_acquire_lock.h remove copyright years (#3286) 2019-12-11 21:14:08 +03:00
citus_clauses.h Rename master evaluation to coordinator evaluation 2020-07-07 10:37:41 +02:00
citus_custom_scan.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
citus_nodefuncs.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
citus_nodes.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
citus_ruleutils.h Implement methods to process & recreate triggers on citus tables 2020-05-31 15:28:17 +03:00
citus_safe_lib.h Semmle: Ensure stack memory is not leaked through uninitialized… (#3561) 2020-03-11 20:05:36 +01:00
colocation_utils.h add a UDF to update colocation (#3623) 2020-03-23 13:22:24 +03:00
combine_query_planner.h Fix some more master->coordinator comments 2020-07-07 10:37:53 +02:00
commands.h Check the removeType in IsDropCitusStmt (#3859) 2020-06-05 20:49:54 +03:00
connection_management.h change wording to allowlist and the likes (#3906) 2020-07-15 16:24:40 +02:00
coordinator_protocol.h Fix some more master->coordinator comments 2020-07-07 10:37:53 +02:00
cte_inline.h Update regression tests-2 2020-01-16 12:28:15 +01:00
deparse_shard_query.h Use TupleDestination API for partitioning in insert/select. 2020-07-17 09:43:46 -07:00
deparser.h Propagate ALTER ROLE .. SET statements 2020-03-27 13:02:48 +03:00
directed_acyclic_graph_execution.h Fill in jobIdList field of DistributedExecution 2020-02-05 17:32:22 +00:00
distributed_deadlock_detection.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
distributed_execution_locks.h Remove real time router executors (#3142) 2019-11-05 12:48:10 +01:00
distributed_planner.h Fix some more master->coordinator comments 2020-07-07 10:37:53 +02:00
distribution_column.h Fix segfault in column_to_column_name 2019-12-01 23:57:25 +01:00
enterprise.h remove copyright years (#3286) 2019-12-11 21:14:08 +03:00
error_codes.h Issue worker messages with the same log level 2020-04-14 21:08:25 +02:00
errormessage.h Introduce GetDistributeObjectOps to organize dispatch of logic dependent on node/object type 2020-01-09 18:24:29 +00:00
extended_op_node_utils.h Fix some more master->coordinator comments 2020-07-07 10:37:53 +02:00
foreign_key_relationship.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
function_call_delegation.h Refactor distributed_planner for better understandability 2020-01-06 14:23:38 +01:00
function_utils.h Semmle: Fix obvious issues (#3502) 2020-02-21 10:16:00 +01:00
hash_helpers.h use macros for pg versions instead of hardcoded values (#3694) 2020-04-01 17:01:52 +03:00
insert_select_executor.h Move more from insert_select_executor to insert_select_planner 2020-06-26 08:08:26 -07:00
insert_select_planner.h Move more from insert_select_executor to insert_select_planner 2020-06-26 08:08:26 -07:00
intermediate_result_pruning.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
intermediate_results.h CTE statistics in EXPLAIN ANALYZE 2020-06-11 02:39:59 -07:00
listutils.h Make bad refactors to foreach_xxx error out 2020-02-27 10:59:45 +01:00
local_executor.h Implement TupleDestination intereface. 2020-06-05 17:47:40 -07:00
local_multi_copy.h apply review items 2020-03-18 09:33:55 +03:00
local_plan_cache.h extract local plan cache related methods into a file (#3667) 2020-03-31 11:11:34 +03:00
lock_graph.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
log_utils.h Issue worker messages with the same log level 2020-04-14 21:08:25 +02:00
maintenanced.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
memutils.h Implementation for asycn FinishConnectionListEstablishment (#2584) 2019-03-22 17:30:42 +01:00
metadata_cache.h Improve performance of metadata cache (#3924) 2020-06-30 12:10:10 +02:00
metadata_sync.h Rename DistTableCacheEntry to CitusTableCacheEntry 2020-03-07 14:08:03 +00:00
metadata_utility.h Fix some more master->coordinator comments 2020-07-07 10:37:53 +02:00
multi_client_executor.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
multi_executor.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
multi_explain.h Fix task->fetchedExplainAnalyzePlan memory issue. 2020-07-07 07:58:02 -07:00
multi_join_order.h Semmle: Check for NULL in some places where it might occur (#3509) 2020-02-27 10:45:29 +01:00
multi_logical_optimizer.h Fix pushdown of constants in aggregate queries 2020-06-30 11:41:16 -07:00
multi_logical_planner.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
multi_partitioning_utils.h Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
multi_physical_planner.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
multi_progress.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
multi_router_planner.h Handle some NULL issues that static analysis found (#4001) 2020-07-09 15:46:42 +02:00
multi_server_executor.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
namespace_utils.h Implement PushOverrideEmptySearchPath (#3874) 2020-06-05 19:23:59 +03:00
pg_dist_colocation.h Add distributioncolumncollation to to pg_dist_colocation 2019-12-09 19:51:40 +00:00
pg_dist_local_group.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
pg_dist_node.h Add shouldhaveshards to pg_dist_node (#2960) 2019-10-22 16:47:16 +02:00
pg_dist_node_metadata.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
pg_dist_partition.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
pg_dist_placement.h Fix some more master->coordinator comments 2020-07-07 10:37:53 +02:00
pg_dist_rebalance_strategy.h Add the necessary changes for rebalance strategies on enterprise (#3325) 2019-12-19 15:23:08 +01:00
pg_dist_shard.h Implement partitioned intermediate results. 2019-12-24 03:53:39 -08:00
pg_dist_transaction.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
pg_version_constants.h use macros for pg versions instead of hardcoded values (#3694) 2020-04-01 17:01:52 +03:00
placement_access.h Remove real time router executors (#3142) 2019-11-05 12:48:10 +01:00
placement_connection.h Locally execute queries that don't need any data access (#3410) 2020-01-23 18:28:34 +01:00
query_colocation_checker.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
query_pushdown_planning.h Introduce intermediate result broadcasting 2019-11-20 15:26:36 +03:00
query_stats.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
query_utils.h Include all relevant relations in the ExtractRangeTableRelationWalker (#3135) 2019-11-01 16:06:58 +01:00
recursive_planning.h use macros for pg versions instead of hardcoded values (#3694) 2020-04-01 17:01:52 +03:00
reference_table_utils.h GUC for replicate reference tables on activate. 2020-04-08 12:42:45 -07:00
relation_access_tracking.h Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
relation_restriction_equivalence.h Fix typos 2020-02-18 17:14:40 +00:00
relay_utility.h Rename RelayFileState to ShardState 2020-01-12 05:57:53 +00:00
remote_commands.h Support use of binary protocol in between nodes (#3877) 2020-06-12 15:02:51 +02:00
remote_transaction.h rename REMOTE_TRANS_INVALID to REMOTE_TRANS_NOT_STARTED 2019-12-11 15:24:18 +03:00
repartition_join_execution.h Add adaptive executor support for repartition joins (#3169) 2019-12-17 19:09:45 +03:00
resource_lock.h Defer reference table replication to shard creation time 2020-04-08 12:41:36 -07:00
run_from_same_connection.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
shard_pruning.h Coerce types properly for INSERT 2020-06-10 10:40:28 +02:00
shard_utils.h implement GetOnlyShardOidOfReferenceTable and some refactor in shard_uitls 2020-04-03 18:24:13 +03:00
shardinterval_utils.h Remove master from file hierarchy 2020-06-16 17:49:09 +02:00
shared_connection_stats.h Remove the entries from the shared connection counter hash when no connections remain (#3775) 2020-04-17 17:14:58 +03:00
shared_library_init.h refactor some of hard coded values in citus gucs (#3137) 2019-10-30 10:35:39 +03:00
statistics_collection.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
subplan_execution.h Locally execute queries that don't need any data access (#3410) 2020-01-23 18:28:34 +01:00
task_execution_utils.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
tdigest_extension.h Feature: tdigest aggregate (#3897) 2020-06-12 13:50:28 +02:00
time_constants.h refactor some of hard coded values in citus gucs (#3137) 2019-10-30 10:35:39 +03:00
transaction_identifier.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
transaction_management.h Defer reference table replication to shard creation time 2020-04-08 12:41:36 -07:00
transaction_recovery.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
transmit.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
tuple_destination.h Fix EXPLAIN ANALYZE received data counter issues (#3917) 2020-06-17 11:33:38 +02:00
tuplestore.h Remove copyright years (#2918) 2019-10-15 17:44:30 +03:00
version_compat.h use macros for pg versions instead of hardcoded values (#3694) 2020-04-01 17:01:52 +03:00
worker_create_or_replace.h create_distributed_function: accept aggregates 2019-11-06 18:23:37 +00:00
worker_log_messages.h Issue worker messages with the same log level 2020-04-14 21:08:25 +02:00
worker_manager.h rename node/worker utilities (#4003) 2020-07-09 15:30:35 +03:00
worker_protocol.h Remove task tracker executor (#3850) 2020-07-18 13:11:36 +03:00
worker_shard_visibility.h Plan reference/local table joins locally 2019-11-15 07:36:50 -08:00
worker_transaction.h address feedback 2020-07-13 19:45:02 +03:00