From b3af63c8cecd1266704e5af46e31308fdc8af58e Mon Sep 17 00:00:00 2001 From: SaitTalhaNisanci Date: Sat, 18 Jul 2020 13:11:36 +0300 Subject: [PATCH] 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 03ecc0a681e69fe8c88f48498bcaffc4ce44de6f. * 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 --- .circleci/config.yml | 20 +- src/backend/distributed/commands/multi_copy.c | 49 - .../connection/connection_configuration.c | 2 +- .../distributed/executor/adaptive_executor.c | 1 - .../distributed/executor/citus_custom_scan.c | 35 - .../directed_acyclic_graph_execution.c | 1 - .../distributed/executor/multi_executor.c | 42 - .../executor/multi_server_executor.c | 177 - .../executor/multi_task_tracker_executor.c | 2945 ----------------- .../distributed/executor/query_stats.c | 5 - .../executor/repartition_join_execution.c | 5 +- .../distributed/planner/distributed_planner.c | 10 +- .../planner/multi_logical_planner.c | 60 - .../planner/multi_physical_planner.c | 302 +- .../planner/multi_router_planner.c | 1 - .../planner/query_pushdown_planning.c | 11 +- .../distributed/planner/recursive_planning.c | 9 - src/backend/distributed/shared_library_init.c | 115 +- .../distributed/sql/citus--9.4-1--9.5-1.sql | 11 + .../sql/downgrades/citus--9.5-1--9.4-1.sql | 59 +- .../test/distributed_intermediate_results.c | 9 +- .../test/run_from_same_connection.c | 3 +- .../distributed/utils/citus_copyfuncs.c | 21 - .../distributed/utils/citus_nodefuncs.c | 3 - .../distributed/utils/citus_outfuncs.c | 23 - .../distributed/utils/multi_resowner.c | 150 - .../distributed/utils/task_execution_utils.c | 224 ++ src/backend/distributed/worker/task_tracker.c | 1108 ------- .../worker/task_tracker_protocol.c | 396 +-- .../worker/worker_data_fetch_protocol.c | 2 +- .../worker/worker_merge_protocol.c | 142 +- .../worker/worker_partition_protocol.c | 40 +- .../worker/worker_sql_task_protocol.c | 42 +- src/include/distributed/citus_custom_scan.h | 1 - src/include/distributed/citus_nodefuncs.h | 2 - src/include/distributed/citus_nodes.h | 1 - .../distributed/intermediate_result_pruning.h | 1 + .../distributed/multi_client_executor.h | 12 - src/include/distributed/multi_executor.h | 1 - .../distributed/multi_logical_planner.h | 1 - .../distributed/multi_physical_planner.h | 26 +- src/include/distributed/multi_resowner.h | 21 - .../distributed/multi_server_executor.h | 136 +- .../distributed/multi_task_tracker_executor.h | 8 - .../distributed/task_execution_utils.h | 6 + src/include/distributed/task_tracker.h | 135 - .../distributed/task_tracker_protocol.h | 29 - src/include/distributed/worker_protocol.h | 4 +- src/test/regress/Makefile | 5 - .../regress/expected/adaptive_executor.out | 1 - .../adaptive_executor_repartition.out | 1 - .../regress/expected/aggregate_support.out | 13 +- .../expected/isolation_append_copy_vs_all.out | 8 +- .../expected/isolation_hash_copy_vs_all.out | 40 +- .../isolation_partitioned_copy_vs_all.out | 8 +- .../expected/isolation_range_copy_vs_all.out | 8 +- .../isolation_reference_copy_vs_all.out | 6 +- .../expected/isolation_select_vs_all.out | 70 +- .../expected/limit_intermediate_size.out | 38 +- .../expected/local_shard_execution.out | 13 +- .../multi_binary_master_copy_format.out | 2 - .../expected/multi_complex_expressions.out | 2 - .../expected/multi_complex_expressions_0.out | 2 - .../regress/expected/multi_cross_shard.out | 4 +- src/test/regress/expected/multi_explain.out | 18 +- src/test/regress/expected/multi_extension.out | 10 +- .../expected/multi_follower_task_tracker.out | 27 - .../regress/expected/multi_hash_pruning.out | 1 - .../expected/multi_join_order_additional.out | 19 +- .../multi_join_order_tpch_repartition.out | 11 +- src/test/regress/expected/multi_multiuser.out | 164 +- .../regress/expected/multi_mx_explain.out | 12 +- .../expected/multi_mx_reference_table.out | 2 +- .../expected/multi_mx_repartition_join_w1.out | 3 +- .../expected/multi_mx_repartition_join_w2.out | 3 +- .../multi_mx_repartition_udt_prepare.out | 3 +- .../expected/multi_mx_repartition_udt_w1.out | 3 +- .../expected/multi_mx_repartition_udt_w2.out | 3 +- .../expected/multi_mx_router_planner.out | 5 +- .../expected/multi_mx_schema_support.out | 11 +- .../multi_null_minmax_value_pruning.out | 1 - .../regress/expected/multi_partitioning.out | 3 +- .../regress/expected/multi_prepare_plsql.out | 4 - .../regress/expected/multi_prepare_sql.out | 4 - .../expected/multi_reference_table.out | 9 +- .../multi_repartition_join_planning.out | 231 +- .../multi_repartition_join_pruning.out | 24 +- ...multi_repartition_join_task_assignment.out | 2 +- .../expected/multi_repartition_udt.out | 4 +- .../multi_repartitioned_subquery_udf.out | 1 - .../regress/expected/multi_router_planner.out | 52 +- .../multi_router_planner_fast_path.out | 2 - .../regress/expected/multi_schema_support.out | 4 +- .../regress/expected/multi_simple_queries.out | 1 - .../expected/multi_simple_queries_0.out | 1 - .../multi_single_relation_subquery.out | 19 +- .../regress/expected/multi_sql_function.out | 2 - .../expected/multi_task_string_size.out | 10 +- src/test/regress/expected/multi_view.out | 27 +- .../expected/propagate_set_commands.out | 1 - .../set_operation_and_local_tables.out | 3 +- src/test/regress/expected/single_node.out | 6 +- .../expected/task_tracker_assign_task.out | 89 - .../expected/task_tracker_cleanup_job.out | 110 - .../expected/task_tracker_create_table.out | 11 - .../expected/task_tracker_partition_task.out | 108 - src/test/regress/expected/with_executors.out | 6 +- .../worker_check_invalid_arguments.out | 100 - .../regress/expected/worker_remove_files.out | 8 - .../input/multi_complex_count_distinct.source | 2 - src/test/regress/multi_follower_schedule | 1 - src/test/regress/multi_mx_schedule | 3 +- .../regress/multi_task_tracker_extra_schedule | 123 - .../multi_complex_count_distinct.source | 286 +- src/test/regress/pg_regress_multi.pl | 1 + .../spec/isolation_append_copy_vs_all.spec | 4 +- .../spec/isolation_hash_copy_vs_all.spec | 4 +- .../isolation_partitioned_copy_vs_all.spec | 4 +- .../spec/isolation_range_copy_vs_all.spec | 4 +- .../spec/isolation_reference_copy_vs_all.spec | 6 +- .../regress/spec/isolation_select_vs_all.spec | 4 +- src/test/regress/sql/adaptive_executor.sql | 1 - .../sql/adaptive_executor_repartition.sql | 1 - src/test/regress/sql/aggregate_support.sql | 5 +- .../regress/sql/limit_intermediate_size.sql | 5 +- .../sql/multi_binary_master_copy_format.sql | 2 - .../regress/sql/multi_complex_expressions.sql | 2 - src/test/regress/sql/multi_cross_shard.sql | 5 +- src/test/regress/sql/multi_explain.sql | 4 +- .../sql/multi_follower_task_tracker.sql | 22 - src/test/regress/sql/multi_hash_pruning.sql | 1 - .../sql/multi_join_order_additional.sql | 1 + .../sql/multi_join_order_tpch_repartition.sql | 1 + src/test/regress/sql/multi_multiuser.sql | 78 +- src/test/regress/sql/multi_mx_explain.sql | 2 - .../regress/sql/multi_mx_reference_table.sql | 2 +- .../sql/multi_mx_repartition_join_w1.sql | 3 +- .../sql/multi_mx_repartition_join_w2.sql | 3 +- .../sql/multi_mx_repartition_udt_prepare.sql | 1 + .../sql/multi_mx_repartition_udt_w1.sql | 3 +- .../sql/multi_mx_repartition_udt_w2.sql | 3 +- .../regress/sql/multi_mx_router_planner.sql | 5 +- .../regress/sql/multi_mx_schema_support.sql | 4 +- .../sql/multi_null_minmax_value_pruning.sql | 1 - src/test/regress/sql/multi_partitioning.sql | 3 +- src/test/regress/sql/multi_prepare_plsql.sql | 4 - src/test/regress/sql/multi_prepare_sql.sql | 4 - .../regress/sql/multi_reference_table.sql | 10 +- .../sql/multi_repartition_join_planning.sql | 4 +- .../sql/multi_repartition_join_pruning.sql | 2 +- ...multi_repartition_join_task_assignment.sql | 2 +- .../regress/sql/multi_repartition_udt.sql | 2 + .../sql/multi_repartitioned_subquery_udf.sql | 1 - src/test/regress/sql/multi_router_planner.sql | 15 +- .../sql/multi_router_planner_fast_path.sql | 2 - src/test/regress/sql/multi_schema_support.sql | 4 +- src/test/regress/sql/multi_simple_queries.sql | 1 - .../sql/multi_single_relation_subquery.sql | 1 - src/test/regress/sql/multi_sql_function.sql | 2 - .../regress/sql/multi_task_string_size.sql | 3 +- src/test/regress/sql/multi_view.sql | 1 - .../regress/sql/propagate_set_commands.sql | 1 - .../sql/set_operation_and_local_tables.sql | 4 +- .../regress/sql/task_tracker_assign_task.sql | 51 - .../regress/sql/task_tracker_cleanup_job.sql | 51 - .../regress/sql/task_tracker_create_table.sql | 17 - .../sql/task_tracker_partition_task.sql | 69 - .../sql/worker_check_invalid_arguments.sql | 105 - src/test/regress/sql/worker_remove_files.sql | 5 - src/test/regress/worker_schedule | 13 +- 170 files changed, 1011 insertions(+), 7729 deletions(-) delete mode 100644 src/backend/distributed/executor/multi_task_tracker_executor.c delete mode 100644 src/backend/distributed/utils/multi_resowner.c create mode 100644 src/backend/distributed/utils/task_execution_utils.c delete mode 100644 src/backend/distributed/worker/task_tracker.c delete mode 100644 src/include/distributed/multi_resowner.h delete mode 100644 src/include/distributed/multi_task_tracker_executor.h create mode 100644 src/include/distributed/task_execution_utils.h delete mode 100644 src/include/distributed/task_tracker.h delete mode 100644 src/include/distributed/task_tracker_protocol.h delete mode 100644 src/test/regress/expected/multi_follower_task_tracker.out delete mode 100644 src/test/regress/expected/worker_check_invalid_arguments.out delete mode 100644 src/test/regress/multi_task_tracker_extra_schedule delete mode 100644 src/test/regress/sql/multi_follower_task_tracker.sql delete mode 100644 src/test/regress/sql/task_tracker_assign_task.sql delete mode 100644 src/test/regress/sql/task_tracker_cleanup_job.sql delete mode 100644 src/test/regress/sql/task_tracker_create_table.sql delete mode 100644 src/test/regress/sql/task_tracker_partition_task.sql delete mode 100644 src/test/regress/sql/worker_check_invalid_arguments.sql delete mode 100644 src/test/regress/sql/worker_remove_files.sql diff --git a/.circleci/config.yml b/.circleci/config.yml index 7287579e2..84050db1c 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -81,7 +81,7 @@ jobs: - codecov/upload: flags: 'test_11,multi' - test-11_check-tt-van-mx: + test-11_check-van-mx: docker: - image: 'citus/exttester-11:latest' working_directory: /home/circleci/project @@ -89,11 +89,11 @@ jobs: - attach_workspace: at: . - run: - name: 'Install and Test (check-tt-van-mx)' - command: 'chown -R circleci:circleci /home/circleci && install-and-test-ext check-multi-task-tracker-extra check-vanilla check-multi-mx' + name: 'Install and Test (check-van-mx)' + command: 'chown -R circleci:circleci /home/circleci && install-and-test-ext check-vanilla check-multi-mx' no_output_timeout: 2m - codecov/upload: - flags: 'test_11,tracker,vanilla,mx' + flags: 'test_11,vanilla,mx' test-11_check-iso-work-fol: docker: - image: 'citus/exttester-11:latest' @@ -167,7 +167,7 @@ jobs: no_output_timeout: 2m - codecov/upload: flags: 'test_12,multi' - test-12_check-tt-van-mx: + test-12_check-van-mx: docker: - image: 'citus/exttester-12:latest' working_directory: /home/circleci/project @@ -175,11 +175,11 @@ jobs: - attach_workspace: at: . - run: - name: 'Install and Test (check-tt-van-mx)' - command: 'chown -R circleci:circleci /home/circleci && install-and-test-ext check-multi-task-tracker-extra check-vanilla check-multi-mx' + name: 'Install and Test (check-van-mx)' + command: 'chown -R circleci:circleci /home/circleci && install-and-test-ext check-vanilla check-multi-mx' no_output_timeout: 2m - codecov/upload: - flags: 'test_12,tracker,vanilla,mx' + flags: 'test_12,vanilla,mx' test-12_check-iso-work-fol: docker: - image: 'citus/exttester-12:latest' @@ -305,7 +305,7 @@ workflows: - test-11_check-multi: requires: [build] - - test-11_check-tt-van-mx: + - test-11_check-van-mx: requires: [build] - test-11_check-iso-work-fol: requires: [build] @@ -316,7 +316,7 @@ workflows: - test-12_check-multi: requires: [build] - - test-12_check-tt-van-mx: + - test-12_check-van-mx: requires: [build] - test-12_check-iso-work-fol: requires: [build] diff --git a/src/backend/distributed/commands/multi_copy.c b/src/backend/distributed/commands/multi_copy.c index e2a3491c4..86dc282b0 100644 --- a/src/backend/distributed/commands/multi_copy.c +++ b/src/backend/distributed/commands/multi_copy.c @@ -2842,55 +2842,6 @@ ProcessCopyStmt(CopyStmt *copyStatement, char *completionTag, const char *queryS } } } - - - if (copyStatement->filename != NULL && !copyStatement->is_program) - { - char *filename = copyStatement->filename; - - /* - * We execute COPY commands issued by the task-tracker executor here - * because we're not normally allowed to write to a file as a regular - * user and we don't want to execute the query as superuser. - */ - if (CacheDirectoryElement(filename) && copyStatement->query != NULL && - !copyStatement->is_from && !is_absolute_path(filename)) - { - bool binaryCopyFormat = CopyStatementHasFormat(copyStatement, "binary"); - Query *query = NULL; - Node *queryNode = copyStatement->query; - StringInfo userFilePath = makeStringInfo(); - - RawStmt *rawStmt = makeNode(RawStmt); - rawStmt->stmt = queryNode; - - List *queryTreeList = pg_analyze_and_rewrite(rawStmt, queryString, NULL, 0, - NULL); - - if (list_length(queryTreeList) != 1) - { - ereport(ERROR, (errmsg("can only execute a single query"))); - } - - query = (Query *) linitial(queryTreeList); - - /* - * Add a user ID suffix to prevent other users from reading/writing - * the same file. We do this consistently in all functions that interact - * with task files. - */ - appendStringInfo(userFilePath, "%s.%u", filename, GetUserId()); - - int64 tuplesSent = WorkerExecuteSqlTask(query, filename, binaryCopyFormat); - - SafeSnprintf(completionTag, COMPLETION_TAG_BUFSIZE, - "COPY " UINT64_FORMAT, tuplesSent); - - return NULL; - } - } - - return (Node *) copyStatement; } diff --git a/src/backend/distributed/connection/connection_configuration.c b/src/backend/distributed/connection/connection_configuration.c index 38225bc2a..be0e85676 100644 --- a/src/backend/distributed/connection/connection_configuration.c +++ b/src/backend/distributed/connection/connection_configuration.c @@ -14,7 +14,7 @@ #include "distributed/connection_management.h" #include "distributed/metadata_cache.h" #include "distributed/worker_manager.h" -#include "distributed/task_tracker.h" + #include "postmaster/postmaster.h" #include "mb/pg_wchar.h" #include "utils/builtins.h" diff --git a/src/backend/distributed/executor/adaptive_executor.c b/src/backend/distributed/executor/adaptive_executor.c index bc46b5364..a22a5da75 100644 --- a/src/backend/distributed/executor/adaptive_executor.c +++ b/src/backend/distributed/executor/adaptive_executor.c @@ -148,7 +148,6 @@ #include "distributed/multi_explain.h" #include "distributed/multi_partitioning_utils.h" #include "distributed/multi_physical_planner.h" -#include "distributed/multi_resowner.h" #include "distributed/multi_server_executor.h" #include "distributed/placement_access.h" #include "distributed/placement_connection.h" diff --git a/src/backend/distributed/executor/citus_custom_scan.c b/src/backend/distributed/executor/citus_custom_scan.c index e3d2e5636..c1abeeeaa 100644 --- a/src/backend/distributed/executor/citus_custom_scan.c +++ b/src/backend/distributed/executor/citus_custom_scan.c @@ -48,7 +48,6 @@ /* functions for creating custom scan nodes */ static Node * AdaptiveExecutorCreateScan(CustomScan *scan); -static Node * TaskTrackerCreateScan(CustomScan *scan); static Node * NonPushableInsertSelectCreateScan(CustomScan *scan); static Node * DelayedErrorCreateScan(CustomScan *scan); @@ -72,11 +71,6 @@ CustomScanMethods AdaptiveExecutorCustomScanMethods = { AdaptiveExecutorCreateScan }; -CustomScanMethods TaskTrackerCustomScanMethods = { - "Citus Task-Tracker", - TaskTrackerCreateScan -}; - CustomScanMethods NonPushableInsertSelectCustomScanMethods = { "Citus INSERT ... SELECT", NonPushableInsertSelectCreateScan @@ -100,15 +94,6 @@ static CustomExecMethods AdaptiveExecutorCustomExecMethods = { .ExplainCustomScan = CitusExplainScan }; -static CustomExecMethods TaskTrackerCustomExecMethods = { - .CustomName = "TaskTrackerScan", - .BeginCustomScan = CitusBeginScan, - .ExecCustomScan = TaskTrackerExecScan, - .EndCustomScan = CitusEndScan, - .ReScanCustomScan = CitusReScan, - .ExplainCustomScan = CitusExplainScan -}; - static CustomExecMethods NonPushableInsertSelectCustomExecMethods = { .CustomName = "NonPushableInsertSelectScan", .BeginCustomScan = CitusBeginScan, @@ -132,7 +117,6 @@ IsCitusCustomState(PlanState *planState) CustomScanState *css = castNode(CustomScanState, planState); if (css->methods == &AdaptiveExecutorCustomExecMethods || - css->methods == &TaskTrackerCustomExecMethods || css->methods == &NonPushableInsertSelectCustomExecMethods) { return true; @@ -149,7 +133,6 @@ void RegisterCitusCustomScanMethods(void) { RegisterCustomScanMethods(&AdaptiveExecutorCustomScanMethods); - RegisterCustomScanMethods(&TaskTrackerCustomScanMethods); RegisterCustomScanMethods(&NonPushableInsertSelectCustomScanMethods); RegisterCustomScanMethods(&DelayedErrorCustomScanMethods); } @@ -579,24 +562,6 @@ AdaptiveExecutorCreateScan(CustomScan *scan) } -/* - * TaskTrackerCreateScan creates the scan state for task-tracker executor queries. - */ -static Node * -TaskTrackerCreateScan(CustomScan *scan) -{ - CitusScanState *scanState = palloc0(sizeof(CitusScanState)); - - scanState->executorType = MULTI_EXECUTOR_TASK_TRACKER; - scanState->customScanState.ss.ps.type = T_CustomScanState; - scanState->distributedPlan = GetDistributedPlan(scan); - - scanState->customScanState.methods = &TaskTrackerCustomExecMethods; - - return (Node *) scanState; -} - - /* * NonPushableInsertSelectCrateScan creates the scan state for executing * INSERT..SELECT into a distributed table via the coordinator. diff --git a/src/backend/distributed/executor/directed_acyclic_graph_execution.c b/src/backend/distributed/executor/directed_acyclic_graph_execution.c index bd9fa4011..2245db420 100644 --- a/src/backend/distributed/executor/directed_acyclic_graph_execution.c +++ b/src/backend/distributed/executor/directed_acyclic_graph_execution.c @@ -17,7 +17,6 @@ #include "distributed/metadata_cache.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_server_executor.h" -#include "distributed/multi_task_tracker_executor.h" #include "distributed/transaction_management.h" #include "distributed/transmit.h" #include "distributed/worker_manager.h" diff --git a/src/backend/distributed/executor/multi_executor.c b/src/backend/distributed/executor/multi_executor.c index b89da1af8..c4657ccf5 100644 --- a/src/backend/distributed/executor/multi_executor.c +++ b/src/backend/distributed/executor/multi_executor.c @@ -29,7 +29,6 @@ #include "distributed/combine_query_planner.h" #include "distributed/distributed_planner.h" #include "distributed/multi_router_planner.h" -#include "distributed/multi_resowner.h" #include "distributed/multi_server_executor.h" #include "distributed/resource_lock.h" #include "distributed/transaction_management.h" @@ -367,47 +366,6 @@ ReturnTupleFromTuplestore(CitusScanState *scanState) } -/* - * Load data collected by task-tracker executor into the tuplestore - * of CitusScanState. For that, we first create a tuple store, and then copy the - * files one-by-one into the tuple store. - * - * Note that in the long term it'd be a lot better if Multi*Execute() directly - * filled the tuplestores, but that's a fair bit of work. - */ -void -LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob) -{ - List *workerTaskList = workerJob->taskList; - bool randomAccess = true; - bool interTransactions = false; - char *copyFormat = "text"; - - TupleDesc tupleDescriptor = ScanStateGetTupleDescriptor(citusScanState); - - Assert(citusScanState->tuplestorestate == NULL); - citusScanState->tuplestorestate = - tuplestore_begin_heap(randomAccess, interTransactions, work_mem); - - if (BinaryMasterCopyFormat) - { - copyFormat = "binary"; - } - - Task *workerTask = NULL; - foreach_ptr(workerTask, workerTaskList) - { - StringInfo jobDirectoryName = MasterJobDirectoryName(workerTask->jobId); - StringInfo taskFilename = TaskFilename(jobDirectoryName, workerTask->taskId); - - ReadFileIntoTupleStore(taskFilename->data, copyFormat, tupleDescriptor, - citusScanState->tuplestorestate); - } - - tuplestore_donestoring(citusScanState->tuplestorestate); -} - - /* * ReadFileIntoTupleStore parses the records in a COPY-formatted file according * according to the given tuple descriptor and stores the records in a tuple diff --git a/src/backend/distributed/executor/multi_server_executor.c b/src/backend/distributed/executor/multi_server_executor.c index 6bbdc02de..6d5842d62 100644 --- a/src/backend/distributed/executor/multi_server_executor.c +++ b/src/backend/distributed/executor/multi_server_executor.c @@ -24,7 +24,6 @@ #include "distributed/multi_client_executor.h" #include "distributed/multi_executor.h" #include "distributed/multi_physical_planner.h" -#include "distributed/multi_resowner.h" #include "distributed/multi_server_executor.h" #include "distributed/coordinator_protocol.h" #include "distributed/subplan_execution.h" @@ -33,12 +32,9 @@ int RemoteTaskCheckInterval = 100; /* per cycle sleep interval in millisecs */ int TaskExecutorType = MULTI_EXECUTOR_ADAPTIVE; /* distributed executor type */ -bool BinaryMasterCopyFormat = false; /* copy data from workers in binary format */ bool EnableRepartitionJoins = false; -static bool HasReplicatedDistributedTable(List *relationOids); - /* * JobExecutorType selects the executor type for the given distributedPlan using the task * executor type config value. The function then checks if the given distributedPlan needs @@ -109,187 +105,14 @@ JobExecutorType(DistributedPlan *distributedPlan) errhint("Set citus.enable_repartition_joins to on " "to enable repartitioning"))); } - if (HasReplicatedDistributedTable(distributedPlan->relationIdList)) - { - return MULTI_EXECUTOR_TASK_TRACKER; - } return MULTI_EXECUTOR_ADAPTIVE; } } - else - { - int workerNodeCount = list_length(ActiveReadableNodeList()); - int taskCount = list_length(job->taskList); - double tasksPerNode = taskCount / ((double) workerNodeCount); - - /* if we have more tasks per node than what can be tracked, warn the user */ - if (tasksPerNode >= MaxTrackedTasksPerNode) - { - ereport(WARNING, (errmsg("this query assigns more tasks per node than the " - "configured max_tracked_tasks_per_node limit"))); - } - } return executorType; } -/* - * HasReplicatedDistributedTable returns true if there is any - * table in the given list that is: - * - not a reference table - * - has replication factor > 1 - */ -static bool -HasReplicatedDistributedTable(List *relationOids) -{ - Oid oid; - foreach_oid(oid, relationOids) - { - char partitionMethod = PartitionMethod(oid); - if (partitionMethod == DISTRIBUTE_BY_NONE) - { - continue; - } - uint32 tableReplicationFactor = TableShardReplicationFactor(oid); - if (tableReplicationFactor > 1) - { - return true; - } - } - return false; -} - - -/* - * RemoveJobDirectory gets automatically called at portal drop (end of query) or - * at transaction abort. The function removes the job directory and releases the - * associated job resource from the resource manager. - */ -void -RemoveJobDirectory(uint64 jobId) -{ - StringInfo jobDirectoryName = MasterJobDirectoryName(jobId); - CitusRemoveDirectory(jobDirectoryName->data); - - ResourceOwnerForgetJobDirectory(CurrentResourceOwner, jobId); -} - - -/* - * InitTaskExecution creates a task execution structure for the given task, and - * initializes execution related fields. - */ -TaskExecution * -InitTaskExecution(Task *task, TaskExecStatus initialTaskExecStatus) -{ - /* each task placement (assignment) corresponds to one worker node */ - uint32 nodeCount = list_length(task->taskPlacementList); - - TaskExecution *taskExecution = CitusMakeNode(TaskExecution); - - taskExecution->jobId = task->jobId; - taskExecution->taskId = task->taskId; - taskExecution->nodeCount = nodeCount; - taskExecution->currentNodeIndex = 0; - taskExecution->failureCount = 0; - - taskExecution->taskStatusArray = palloc0(nodeCount * sizeof(TaskExecStatus)); - taskExecution->transmitStatusArray = palloc0(nodeCount * sizeof(TransmitExecStatus)); - taskExecution->connectionIdArray = palloc0(nodeCount * sizeof(int32)); - taskExecution->fileDescriptorArray = palloc0(nodeCount * sizeof(int32)); - - for (uint32 nodeIndex = 0; nodeIndex < nodeCount; nodeIndex++) - { - taskExecution->taskStatusArray[nodeIndex] = initialTaskExecStatus; - taskExecution->transmitStatusArray[nodeIndex] = EXEC_TRANSMIT_UNASSIGNED; - taskExecution->connectionIdArray[nodeIndex] = INVALID_CONNECTION_ID; - taskExecution->fileDescriptorArray[nodeIndex] = -1; - } - - return taskExecution; -} - - -/* - * CleanupTaskExecution iterates over all connections and file descriptors for - * the given task execution. The function first closes all open connections and - * file descriptors, and then frees memory allocated for the task execution. - */ -void -CleanupTaskExecution(TaskExecution *taskExecution) -{ - for (uint32 nodeIndex = 0; nodeIndex < taskExecution->nodeCount; nodeIndex++) - { - int32 connectionId = taskExecution->connectionIdArray[nodeIndex]; - int32 fileDescriptor = taskExecution->fileDescriptorArray[nodeIndex]; - - /* close open connection */ - if (connectionId != INVALID_CONNECTION_ID) - { - MultiClientDisconnect(connectionId); - taskExecution->connectionIdArray[nodeIndex] = INVALID_CONNECTION_ID; - } - - /* close open file */ - if (fileDescriptor >= 0) - { - int closed = close(fileDescriptor); - taskExecution->fileDescriptorArray[nodeIndex] = -1; - - if (closed < 0) - { - ereport(WARNING, (errcode_for_file_access(), - errmsg("could not close copy file: %m"))); - } - } - } - - /* deallocate memory and reset all fields */ - pfree(taskExecution->taskStatusArray); - pfree(taskExecution->connectionIdArray); - pfree(taskExecution->fileDescriptorArray); - pfree(taskExecution); -} - - -/* Determines if the given task exceeded its failure threshold. */ -bool -TaskExecutionFailed(TaskExecution *taskExecution) -{ - if (taskExecution->failureCount >= MAX_TASK_EXECUTION_FAILURES) - { - return true; - } - - return false; -} - - -/* - * AdjustStateForFailure increments the failure count for given task execution. - * The function also determines the next worker node that should be contacted - * for remote execution. - */ -void -AdjustStateForFailure(TaskExecution *taskExecution) -{ - int maxNodeIndex = taskExecution->nodeCount - 1; - Assert(maxNodeIndex >= 0); - - if (taskExecution->currentNodeIndex < maxNodeIndex) - { - taskExecution->currentNodeIndex++; /* try next worker node */ - } - else - { - taskExecution->currentNodeIndex = 0; /* go back to the first worker node */ - } - - taskExecution->failureCount++; /* record failure */ -} - - /* * CheckIfSizeLimitIsExceeded checks if the limit is exceeded by intermediate * results, if there is any. diff --git a/src/backend/distributed/executor/multi_task_tracker_executor.c b/src/backend/distributed/executor/multi_task_tracker_executor.c deleted file mode 100644 index a9df07251..000000000 --- a/src/backend/distributed/executor/multi_task_tracker_executor.c +++ /dev/null @@ -1,2945 +0,0 @@ -/*------------------------------------------------------------------------- - * - * multi_task_tracker_executor.c - * - * Routines for executing remote tasks as part of a distributed execution plan - * using task trackers. These task trackers receive task assignments from this - * executor, and they manage task executions on worker nodes. The use of task - * trackers brings us two benefits: (a) distributed execution plans can scale - * out to many tasks, as the executor no longer needs to keep a connection open - * for each task, and (b) distributed execution plans can include map/reduce - * execution primitives, which involve writing intermediate results to files. - * - * Copyright (c) Citus Data, Inc. - * - * $Id$ - * - *------------------------------------------------------------------------- - */ - -#include "postgres.h" -#include "miscadmin.h" - -#include -#include -#include - -#include "commands/dbcommands.h" -#include "distributed/citus_custom_scan.h" -#include "distributed/citus_nodes.h" -#include "distributed/connection_management.h" -#include "distributed/deparse_shard_query.h" -#include "distributed/distributed_execution_locks.h" -#include "distributed/listutils.h" -#include "distributed/local_executor.h" -#include "distributed/metadata_cache.h" -#include "distributed/multi_client_executor.h" -#include "distributed/multi_executor.h" -#include "distributed/multi_physical_planner.h" -#include "distributed/multi_server_executor.h" -#include "distributed/multi_resowner.h" -#include "distributed/pg_dist_partition.h" -#include "distributed/resource_lock.h" -#include "distributed/subplan_execution.h" -#include "distributed/worker_protocol.h" -#include "distributed/version_compat.h" -#include "distributed/multi_task_tracker_executor.h" -#include "storage/fd.h" -#include "utils/builtins.h" -#include "utils/hsearch.h" -#include "utils/timestamp.h" - - -int MaxAssignTaskBatchSize = 64; /* maximum number of tasks to assign per round */ -int MaxTaskStatusBatchSize = 64; /* maximum number of tasks status checks per round */ - - -/* TaskMapKey is used as a key in task hash */ -typedef struct TaskMapKey -{ - TaskType taskType; - uint64 jobId; - uint32 taskId; -} TaskMapKey; - - -/* - * TaskMapEntry is used as entry in task hash. We need to keep a pointer - * of the task in the entry. - */ -typedef struct TaskMapEntry -{ - TaskMapKey key; - Task *task; -} TaskMapEntry; - - -/* Local functions forward declarations to init tasks and trackers */ - -static HTAB * TaskHashCreate(uint32 taskHashSize); -static Task * TaskHashEnter(HTAB *taskHash, Task *task); -static Task * TaskHashLookup(HTAB *trackerHash, TaskType taskType, uint64 jobId, - uint32 taskId); -static bool TopLevelTask(Task *task); -static bool TransmitExecutionCompleted(TaskExecution *taskExecution); -static HTAB * TrackerHash(const char *taskTrackerHashName, List *workerNodeList, - char *userName); -static HTAB * TrackerHashCreate(const char *taskTrackerHashName, - uint32 taskTrackerHashSize); -static TaskTracker * TrackerHashEnter(HTAB *taskTrackerHash, char *nodeName, - uint32 nodePort); -static void TrackerHashConnect(HTAB *taskTrackerHash); -static TrackerStatus TrackerConnectPoll(TaskTracker *taskTracker); -static TaskTracker * ResolveTaskTracker(HTAB *trackerHash, Task *task, - TaskExecution *taskExecution); -static TaskTracker * ResolveMapTaskTracker(HTAB *trackerHash, Task *task, - TaskExecution *taskExecution); -static TaskTracker * TrackerHashLookup(HTAB *trackerHash, const char *nodeName, - uint32 nodePort); -static void PrepareMasterJobDirectory(Job *workerJob); - -/* Local functions forward declarations to manage tasks and their assignments */ -static TaskExecStatus ManageTaskExecution(TaskTracker *taskTracker, - TaskTracker *sourceTaskTracker, - Task *task, TaskExecution *taskExecution); -static TransmitExecStatus ManageTransmitExecution(TaskTracker *transmitTracker, - Task *task, - TaskExecution *taskExecution, - DistributedExecutionStats * - executionStats); -static bool TaskExecutionsCompleted(List *taskList); -static StringInfo MapFetchTaskQueryString(Task *mapFetchTask, Task *mapTask); -static void TrackerQueueSqlTask(TaskTracker *taskTracker, Task *task); -static void TrackerQueueTask(TaskTracker *taskTracker, Task *task); -static StringInfo TaskAssignmentQuery(Task *task, char *queryString); -static TaskStatus TrackerTaskStatus(TaskTracker *taskTracker, Task *task); -static TrackerTaskState * TrackerTaskStateHashLookup(HTAB *taskStateHash, Task *task); -static bool TrackerHealthy(TaskTracker *taskTracker); -static void TrackerQueueFileTransmit(TaskTracker *transmitTracker, Task *task); -static TrackerTaskState * TaskStateHashEnter(HTAB *taskStateHash, uint64 jobId, - uint32 taskId); -static int32 TransmitTrackerConnectionId(TaskTracker *transmitTracker, Task *task); - -/* Local functions forward declarations to manage task failovers */ -static List * ConstrainedTaskList(List *taskAndExecutionList, Task *task); -static List * ConstrainedNonMergeTaskList(List *taskAndExecutionList, Task *task); -static List * UpstreamDependencyList(List *taskAndExecutionList, Task *searchedTask); -static List * ConstrainedMergeTaskList(List *taskAndExecutionList, Task *task); -static List * MergeTaskList(List *taskList); -static void ReassignTaskList(List *taskList); -static void ReassignMapFetchTaskList(List *mapFetchTaskList); - -/* Local functions forward declarations to manage task trackers */ -static void ManageTaskTracker(TaskTracker *taskTracker); -static bool TrackerConnectionUp(TaskTracker *taskTracker); -static void TrackerReconnectPoll(TaskTracker *taskTracker); -static List * AssignQueuedTasks(TaskTracker *taskTracker); -static List * TaskStatusBatchList(TaskTracker *taskTracker); -static StringInfo TaskStatusBatchQuery(List *taskList); -static void ReceiveTaskStatusBatchQueryResponse(TaskTracker *taskTracker); -static void ManageTransmitTracker(TaskTracker *transmitTracker); -static TrackerTaskState * NextQueuedFileTransmit(HTAB *taskStateHash); - -/* Local functions forward declarations to clean up tasks */ -static List * JobIdList(Job *job); -static void TrackerCleanupResources(HTAB *taskTrackerHash, HTAB *transmitTrackerHash, - List *jobIdList, List *taskList); -static void TrackerHashWaitActiveRequest(HTAB *taskTrackerHash); -static void TrackerHashCancelActiveRequest(HTAB *taskTrackerHash); -static Task * JobCleanupTask(uint64 jobId); -static void TrackerHashCleanupJob(HTAB *taskTrackerHash, Task *jobCleanupTask); -static void TrackerHashDisconnect(HTAB *taskTrackerHash); - - -/* - * MultiTaskTrackerExecute loops over given tasks, and manages their execution - * until either one task permanently fails or all tasks successfully complete. - * The function initializes connections to task trackers on worker nodes, and - * executes tasks through assigning them to these trackers. - */ -void -MultiTaskTrackerExecute(Job *job) -{ - List *jobTaskList = job->taskList; - uint32 topLevelTaskCount = 0; - uint32 failedTaskId = 0; - bool allTasksCompleted = false; - bool taskFailed = false; - bool taskTransmitFailed = false; - bool clusterFailed = false; - bool sizeLimitIsExceeded = false; - - DistributedExecutionStats executionStats = { 0 }; - char *extensionOwner = CitusExtensionOwnerName(); - const char *taskTrackerHashName = "Task Tracker Hash"; - const char *transmitTrackerHashName = "Transmit Tracker Hash"; - - if (ReadFromSecondaries == USE_SECONDARY_NODES_ALWAYS) - { - ereport(ERROR, (errmsg("task tracker queries are not allowed while " - "citus.use_secondary_nodes is 'always'"), - errhint("try setting citus.task_executor_type TO 'adaptive'"))); - } - - /* - * We walk over the task tree, and create a task execution struct for each - * task. We then associate the task with its execution and get back a list. - */ - bool createTaskExecution = true; - List *taskAndExecutionList = CreateTaskListForJobTree(jobTaskList, - createTaskExecution); - - /* - * We now count the number of "top level" tasks in the query tree. Once they - * complete, we'll need to fetch these tasks' results to the master node. - */ - Task *task = NULL; - foreach_ptr(task, taskAndExecutionList) - { - bool topLevelTask = TopLevelTask(task); - if (topLevelTask) - { - topLevelTaskCount++; - } - } - - /* - * We get the list of worker nodes, and then create two hashes to manage our - * connections to these nodes. The first hash manages connections used for - * assigning and checking the status of tasks. The second (temporary) hash - * helps us in fetching results data from worker nodes to the master node. - */ - List *workerNodeList = ActivePrimaryNodeList(ShareLock); - uint32 taskTrackerCount = (uint32) list_length(workerNodeList); - - /* connect as the current user for running queries */ - HTAB *taskTrackerHash = TrackerHash(taskTrackerHashName, workerNodeList, NULL); - - /* connect as the superuser for fetching result files */ - HTAB *transmitTrackerHash = TrackerHash(transmitTrackerHashName, workerNodeList, - extensionOwner); - - TrackerHashConnect(taskTrackerHash); - TrackerHashConnect(transmitTrackerHash); - - /* loop around until all tasks complete, one task fails, or user cancels */ - while (!(allTasksCompleted || taskFailed || taskTransmitFailed || - clusterFailed || QueryCancelPending || sizeLimitIsExceeded)) - { - TaskTracker *taskTracker = NULL; - TaskTracker *transmitTracker = NULL; - HASH_SEQ_STATUS taskStatus; - HASH_SEQ_STATUS transmitStatus; - - uint32 completedTransmitCount = 0; - uint32 healthyTrackerCount = 0; - double acceptableHealthyTrackerCount = 0.0; - - /* first, loop around all tasks and manage them */ - foreach_ptr(task, taskAndExecutionList) - { - TaskExecution *taskExecution = task->taskExecution; - - TaskTracker *execTaskTracker = ResolveTaskTracker(taskTrackerHash, - task, taskExecution); - TaskTracker *mapTaskTracker = ResolveMapTaskTracker(taskTrackerHash, - task, taskExecution); - Assert(execTaskTracker != NULL); - - /* call the function that performs the core task execution logic */ - TaskExecStatus taskExecutionStatus = ManageTaskExecution(execTaskTracker, - mapTaskTracker, - task, taskExecution); - - /* - * If task cannot execute on this task/map tracker, we fail over all - * tasks in the same constraint group to the next task/map tracker. - */ - if (taskExecutionStatus == EXEC_TASK_TRACKER_FAILED) - { - /* mark task tracker as failed, in case it isn't marked already */ - execTaskTracker->trackerFailureCount = MAX_TRACKER_FAILURE_COUNT; - - /* - * We may have already started to transmit task results to the - * master. When we reassign the transmits, we could leave the - * transmit tracker in an invalid state. So, we fail it too. - */ - transmitTracker = ResolveTaskTracker(transmitTrackerHash, - task, taskExecution); - transmitTracker->trackerFailureCount = MAX_TRACKER_FAILURE_COUNT; - - List *taskList = ConstrainedTaskList(taskAndExecutionList, task); - ReassignTaskList(taskList); - } - else if (taskExecutionStatus == EXEC_SOURCE_TASK_TRACKER_FAILED) - { - /* first resolve the map task this map fetch task depends on */ - Task *mapTask = (Task *) linitial(task->dependentTaskList); - Assert(task->taskType == MAP_OUTPUT_FETCH_TASK); - - List *mapFetchTaskList = UpstreamDependencyList(taskAndExecutionList, - mapTask); - ReassignMapFetchTaskList(mapFetchTaskList); - - List *mapTaskList = ConstrainedTaskList(taskAndExecutionList, mapTask); - ReassignTaskList(mapTaskList); - } - - /* - * If this task permanently failed, we first need to manually clean - * out client-side resources for all task executions. We therefore - * record the failure here instead of immediately erroring out. - */ - taskFailed = TaskExecutionFailed(taskExecution); - if (taskFailed) - { - failedTaskId = taskExecution->taskId; - break; - } - } - - /* second, loop around "top level" tasks to fetch their results */ - foreach_ptr(task, taskAndExecutionList) - { - TaskExecution *taskExecution = task->taskExecution; - - /* - * We find the tasks that appear in the top level of the query tree, - * and start fetching their results to the master node. - */ - bool topLevelTask = TopLevelTask(task); - if (!topLevelTask) - { - continue; - } - - TaskTracker *execTransmitTracker = ResolveTaskTracker(transmitTrackerHash, - task, taskExecution); - Assert(execTransmitTracker != NULL); - - /* call the function that fetches results for completed SQL tasks */ - TransmitExecStatus transmitExecutionStatus = ManageTransmitExecution( - execTransmitTracker, - task, - taskExecution, - & - executionStats); - - /* - * If we cannot transmit SQL task's results to the master, we first - * force fail the corresponding task tracker. We then fail over all - * tasks in the constraint group to the next task/transmit tracker. - */ - if (transmitExecutionStatus == EXEC_TRANSMIT_TRACKER_FAILED) - { - taskTracker = ResolveTaskTracker(taskTrackerHash, - task, taskExecution); - taskTracker->trackerFailureCount = MAX_TRACKER_FAILURE_COUNT; - - List *taskList = ConstrainedTaskList(taskAndExecutionList, task); - ReassignTaskList(taskList); - } - - /* if task failed for good, record failure and break out of loop */ - taskTransmitFailed = TaskExecutionFailed(taskExecution); - if (taskTransmitFailed) - { - failedTaskId = taskExecution->taskId; - break; - } - - bool transmitCompleted = TransmitExecutionCompleted(taskExecution); - if (transmitCompleted) - { - completedTransmitCount++; - } - } - - - if (CheckIfSizeLimitIsExceeded(&executionStats)) - { - sizeLimitIsExceeded = true; - break; - } - - /* third, loop around task trackers and manage them */ - hash_seq_init(&taskStatus, taskTrackerHash); - hash_seq_init(&transmitStatus, transmitTrackerHash); - - taskTracker = (TaskTracker *) hash_seq_search(&taskStatus); - while (taskTracker != NULL) - { - bool trackerHealthy = TrackerHealthy(taskTracker); - if (trackerHealthy) - { - healthyTrackerCount++; - } - - ManageTaskTracker(taskTracker); - - taskTracker = (TaskTracker *) hash_seq_search(&taskStatus); - } - - transmitTracker = (TaskTracker *) hash_seq_search(&transmitStatus); - while (transmitTracker != NULL) - { - ManageTransmitTracker(transmitTracker); - - transmitTracker = (TaskTracker *) hash_seq_search(&transmitStatus); - } - - /* if more than half the trackers have failed, mark cluster as failed */ - acceptableHealthyTrackerCount = (double) taskTrackerCount / 2.0; - if (healthyTrackerCount < acceptableHealthyTrackerCount) - { - clusterFailed = true; - } - - /* check if we completed execution; otherwise sleep to avoid tight loop */ - if (completedTransmitCount == topLevelTaskCount) - { - allTasksCompleted = true; - } - else - { - long sleepIntervalPerCycle = RemoteTaskCheckInterval * 1000L; - pg_usleep(sleepIntervalPerCycle); - } - } - - /* - * We prevent cancel/die interrupts until we issue cleanup requests to task - * trackers and close open connections. Note that for the above while loop, - * if the user Ctrl+C's a query and we emit a warning before looping to the - * beginning of the while loop, we will get canceled away before we can hold - * any interrupts. - */ - HOLD_INTERRUPTS(); - - List *jobIdList = JobIdList(job); - - TrackerCleanupResources(taskTrackerHash, transmitTrackerHash, - jobIdList, taskAndExecutionList); - - RESUME_INTERRUPTS(); - - /* - * If we previously broke out of the execution loop due to a task failure or - * user cancellation request, we can now safely emit an error message. - */ - if (sizeLimitIsExceeded) - { - ErrorSizeLimitIsExceeded(); - } - else if (taskFailed) - { - ereport(ERROR, (errmsg("failed to execute task %u", failedTaskId))); - } - else if (clusterFailed) - { - ereport(ERROR, (errmsg("majority of nodes failed"))); - } - else if (QueryCancelPending) - { - CHECK_FOR_INTERRUPTS(); - } -} - - -/* - * CreateTaskListForJobTree visits all tasks in the job tree (by following dependentTaskList), - * starting with the given job's task list. - * If createTaskExecution is set to true, for each visited task, - * the function creates a task execution struct associates the task execution with the task, - * and adds the task and its execution to a list. The function then returns the list. - */ -List * -CreateTaskListForJobTree(List *jobTaskList, bool createTaskExecution) -{ - List *taskAndExecutionList = NIL; - const int topLevelTaskHashSize = 32; - int taskHashSize = list_length(jobTaskList) * topLevelTaskHashSize; - HTAB *taskHash = TaskHashCreate(taskHashSize); - - /* - * We walk over the task tree using breadth-first search. For the search, we - * first queue top level tasks in the task tree. - */ - List *taskQueue = list_copy(jobTaskList); - while (taskQueue != NIL) - { - /* pop first element from the task queue */ - Task *task = (Task *) linitial(taskQueue); - taskQueue = list_delete_first(taskQueue); - - if (createTaskExecution) - { - MemoryContext oldContext = MemoryContextSwitchTo(GetMemoryChunkContext(task)); - - /* create task execution and associate it with task */ - TaskExecution *taskExecution = InitTaskExecution(task, EXEC_TASK_UNASSIGNED); - MemoryContextSwitchTo(oldContext); - task->taskExecution = taskExecution; - } - - taskAndExecutionList = lappend(taskAndExecutionList, task); - - List *dependendTaskList = task->dependentTaskList; - - /* - * Push task node's children into the task queue, if and only if - * they're not already there. As task dependencies have to form a - * directed-acyclic-graph and are processed in a breadth-first search - * we can never re-encounter nodes we've already processed. - * - * While we're checking this, we can also fix the problem that - * copyObject() might have duplicated nodes in the graph - if a node - * isn't pushed to the graph because it is already planned to be - * visited, we can simply replace it with the copy. Note that, here - * we only consider dependend tasks. Since currently top level tasks - * cannot be on any dependend task list, we do not check them for duplicates. - * - * taskHash is used to reduce the complexity of keeping track of - * the tasks that are already encountered. - */ - ListCell *dependentTaskCell = NULL; - foreach(dependentTaskCell, dependendTaskList) - { - Task *dependendTask = lfirst(dependentTaskCell); - Task *dependendTaskInHash = TaskHashLookup(taskHash, - dependendTask->taskType, - dependendTask->jobId, - dependendTask->taskId); - - /* - * If the dependend task encountered for the first time, add it to the hash. - * Also, add this task to the task queue. Note that, we do not need to - * add the tasks to the queue which are already encountered, because - * they are already added to the queue. - */ - if (!dependendTaskInHash) - { - dependendTaskInHash = TaskHashEnter(taskHash, dependendTask); - taskQueue = lappend(taskQueue, dependendTaskInHash); - } - - /* update dependentTaskList element to the one which is in the hash */ - lfirst(dependentTaskCell) = dependendTaskInHash; - } - } - - return taskAndExecutionList; -} - - -/* - * TaskHashCreate allocates memory for a task hash, initializes an - * empty hash, and returns this hash. - */ -static HTAB * -TaskHashCreate(uint32 taskHashSize) -{ - HASHCTL info; - const char *taskHashName = "Task Hash"; - - /* - * Can't create a hashtable of size 0. Normally that shouldn't happen, but - * shard pruning currently can lead to this (Job with 0 Tasks). See #833. - */ - if (taskHashSize == 0) - { - taskHashSize = 2; - } - - memset(&info, 0, sizeof(info)); - info.keysize = sizeof(TaskMapKey); - info.entrysize = sizeof(TaskMapEntry); - info.hash = tag_hash; - info.hcxt = CurrentMemoryContext; - int hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT); - - HTAB *taskHash = hash_create(taskHashName, taskHashSize, &info, hashFlags); - - return taskHash; -} - - -/* - * TaskHashEnter creates a reference to the task entry in the given task - * hash. The function errors-out if the same key exists multiple times. - */ -static Task * -TaskHashEnter(HTAB *taskHash, Task *task) -{ - bool handleFound = false; - - TaskMapKey taskKey; - memset(&taskKey, 0, sizeof(TaskMapKey)); - - taskKey.taskType = task->taskType; - taskKey.jobId = task->jobId; - taskKey.taskId = task->taskId; - - void *hashKey = (void *) &taskKey; - TaskMapEntry *taskInTheHash = (TaskMapEntry *) hash_search(taskHash, hashKey, - HASH_ENTER, - &handleFound); - - /* if same node appears twice, we error-out */ - if (handleFound) - { - ereport(ERROR, (errmsg("multiple entries for task: \"%d:" UINT64_FORMAT ":%u\"", - task->taskType, task->jobId, task->taskId))); - } - - /* save the pointer to the original task in the hash */ - taskInTheHash->task = task; - - return task; -} - - -/* - * TaskHashLookup looks for the tasks that corresponds to the given - * taskType, jobId and taskId, and returns the found task, NULL otherwise. - */ -static Task * -TaskHashLookup(HTAB *taskHash, TaskType taskType, uint64 jobId, uint32 taskId) -{ - Task *task = NULL; - bool handleFound = false; - - TaskMapKey taskKey; - memset(&taskKey, 0, sizeof(TaskMapKey)); - - taskKey.taskType = taskType; - taskKey.jobId = jobId; - taskKey.taskId = taskId; - - void *hashKey = (void *) &taskKey; - TaskMapEntry *taskEntry = (TaskMapEntry *) hash_search(taskHash, hashKey, HASH_FIND, - &handleFound); - - if (taskEntry != NULL) - { - task = taskEntry->task; - } - - return task; -} - - -/* - * TopLevelTask checks if the given task appears at the top level of the task - * tree. In doing this, the function assumes the physical planner creates SQL - * tasks only for the top level job. - */ -static bool -TopLevelTask(Task *task) -{ - bool topLevelTask = false; - - /* - * SQL tasks can only appear at the top level in our query tree. Further, no - * other task type can appear at the top level in our tree. - */ - if (task->taskType == READ_TASK) - { - topLevelTask = true; - } - - return topLevelTask; -} - - -/* Determines if the given transmit task successfully completed executing. */ -static bool -TransmitExecutionCompleted(TaskExecution *taskExecution) -{ - bool completed = false; - - for (uint32 nodeIndex = 0; nodeIndex < taskExecution->nodeCount; nodeIndex++) - { - TransmitExecStatus *transmitStatusArray = taskExecution->transmitStatusArray; - - TransmitExecStatus transmitStatus = transmitStatusArray[nodeIndex]; - if (transmitStatus == EXEC_TRANSMIT_DONE) - { - completed = true; - break; - } - } - - return completed; -} - - -/* - * TrackerHash creates a task tracker hash with the given name. The function - * then inserts one task tracker entry for each node in the given worker node - * list, and initializes state for each task tracker. The userName argument - * indicates which user to connect as. - */ -static HTAB * -TrackerHash(const char *taskTrackerHashName, List *workerNodeList, char *userName) -{ - /* create task tracker hash */ - uint32 taskTrackerHashSize = list_length(workerNodeList); - HTAB *taskTrackerHash = TrackerHashCreate(taskTrackerHashName, taskTrackerHashSize); - - WorkerNode *workerNode = NULL; - foreach_ptr(workerNode, workerNodeList) - { - char *nodeName = workerNode->workerName; - uint32 nodePort = workerNode->workerPort; - - uint32 taskStateCount = 32; - HASHCTL info; - - /* insert task tracker into the tracker hash */ - TaskTracker *taskTracker = TrackerHashEnter(taskTrackerHash, nodeName, nodePort); - - - /* for each task tracker, create hash to track its assigned tasks */ - StringInfo taskStateHashName = makeStringInfo(); - appendStringInfo(taskStateHashName, "Task Tracker \"%s:%u\" Task State Hash", - nodeName, nodePort); - - memset(&info, 0, sizeof(info)); - info.keysize = sizeof(uint64) + sizeof(uint32); - info.entrysize = sizeof(TrackerTaskState); - info.hash = tag_hash; - info.hcxt = CurrentMemoryContext; - int hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT); - - HTAB *taskStateHash = hash_create(taskStateHashName->data, taskStateCount, &info, - hashFlags); - if (taskStateHash == NULL) - { - ereport(FATAL, (errcode(ERRCODE_OUT_OF_MEMORY), - errmsg("could not initialize %s", taskStateHashName->data))); - } - - taskTracker->taskStateHash = taskStateHash; - taskTracker->userName = userName; - } - - return taskTrackerHash; -} - - -/* - * TrackerHashCreate allocates memory for a task tracker hash, initializes an - * empty hash, and returns this hash. - */ -static HTAB * -TrackerHashCreate(const char *taskTrackerHashName, uint32 taskTrackerHashSize) -{ - HASHCTL info; - - memset(&info, 0, sizeof(info)); - info.keysize = WORKER_LENGTH + sizeof(uint32); - info.entrysize = sizeof(TaskTracker); - info.hash = tag_hash; - info.hcxt = CurrentMemoryContext; - int hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT); - - HTAB *taskTrackerHash = hash_create(taskTrackerHashName, taskTrackerHashSize, - &info, hashFlags); - if (taskTrackerHash == NULL) - { - ereport(FATAL, (errcode(ERRCODE_OUT_OF_MEMORY), - errmsg("could not initialize task tracker hash"))); - } - - return taskTrackerHash; -} - - -/* - * TrackerHashEnter creates a new task tracker entry in the given task tracker - * hash, and checks that the task tracker entry has been properly created. Note - * that the caller still needs to set the tracker's task state hash field. - */ -static TaskTracker * -TrackerHashEnter(HTAB *taskTrackerHash, char *nodeName, uint32 nodePort) -{ - bool handleFound = false; - - TaskTracker taskTrackerKey; - memset(&taskTrackerKey, 0, sizeof(TaskTracker)); - strlcpy(taskTrackerKey.workerName, nodeName, WORKER_LENGTH); - taskTrackerKey.workerPort = nodePort; - - void *hashKey = (void *) &taskTrackerKey; - TaskTracker *taskTracker = (TaskTracker *) hash_search(taskTrackerHash, hashKey, - HASH_ENTER, &handleFound); - - /* if same node appears twice, we overwrite previous entry */ - if (handleFound) - { - ereport(WARNING, (errmsg("multiple entries for task tracker: \"%s:%u\"", - nodeName, nodePort))); - } - - /* init task tracker object with zeroed out task tracker key */ - *taskTracker = taskTrackerKey; - taskTracker->trackerStatus = TRACKER_CONNECT_START; - taskTracker->connectionId = INVALID_CONNECTION_ID; - taskTracker->currentTaskIndex = -1; - - return taskTracker; -} - - -/* - * TrackerHashConnect walks over each task tracker in the given hash and tries - * to open an asynchronous connection to it. The function then returns when we - * tried connecting to all task trackers and have either succeeded or failed for - * each one of them. - */ -static void -TrackerHashConnect(HTAB *taskTrackerHash) -{ - uint32 taskTrackerCount = (uint32) hash_get_num_entries(taskTrackerHash); - uint32 triedTrackerCount = 0; - - /* loop until we tried to connect to all task trackers */ - while (triedTrackerCount < taskTrackerCount) - { - HASH_SEQ_STATUS status; - - /* loop over the task tracker hash, and poll all trackers again */ - triedTrackerCount = 0; - hash_seq_init(&status, taskTrackerHash); - - TaskTracker *taskTracker = (TaskTracker *) hash_seq_search(&status); - while (taskTracker != NULL) - { - TrackerStatus trackerStatus = TrackerConnectPoll(taskTracker); - if (trackerStatus == TRACKER_CONNECTED || - trackerStatus == TRACKER_CONNECTION_FAILED) - { - triedTrackerCount++; - } - - taskTracker = (TaskTracker *) hash_seq_search(&status); - } - - /* sleep to avoid tight loop */ - long sleepIntervalPerCycle = RemoteTaskCheckInterval * 1000L; - pg_usleep(sleepIntervalPerCycle); - } -} - - -/* - * TrackerConnectPoll opens an asynchronous connection to the given task tracker - * and polls this connection's status on every call. The function also sets task - * tracker's internal state on success, and returns the most recent status for - * the connection. - */ -static TrackerStatus -TrackerConnectPoll(TaskTracker *taskTracker) -{ - switch (taskTracker->trackerStatus) - { - case TRACKER_CONNECT_START: - { - char *nodeName = taskTracker->workerName; - uint32 nodePort = taskTracker->workerPort; - const char *nodeDatabase = CurrentDatabaseName(); - char *nodeUser = taskTracker->userName; - - int32 connectionId = MultiClientConnectStart(nodeName, nodePort, - nodeDatabase, nodeUser); - if (connectionId != INVALID_CONNECTION_ID) - { - taskTracker->connectionId = connectionId; - taskTracker->trackerStatus = TRACKER_CONNECT_POLL; - } - else - { - taskTracker->trackerStatus = TRACKER_CONNECTION_FAILED; - } - - break; - } - - case TRACKER_CONNECT_POLL: - { - int32 connectionId = taskTracker->connectionId; - - ConnectStatus pollStatus = MultiClientConnectPoll(connectionId); - if (pollStatus == CLIENT_CONNECTION_READY) - { - taskTracker->trackerStatus = TRACKER_CONNECTED; - } - else if (pollStatus == CLIENT_CONNECTION_BUSY || - pollStatus == CLIENT_CONNECTION_BUSY_READ || - pollStatus == CLIENT_CONNECTION_BUSY_WRITE) - { - taskTracker->trackerStatus = TRACKER_CONNECT_POLL; - } - else if (pollStatus == CLIENT_CONNECTION_BAD) - { - taskTracker->trackerStatus = TRACKER_CONNECTION_FAILED; - - MultiClientDisconnect(connectionId); - taskTracker->connectionId = INVALID_CONNECTION_ID; - } - - /* now check if we have been trying to connect for too long */ - taskTracker->connectPollCount++; - if (pollStatus == CLIENT_CONNECTION_BUSY_READ || - pollStatus == CLIENT_CONNECTION_BUSY_WRITE) - { - uint32 maxCount = - ceil(NodeConnectionTimeout * 1.0f / RemoteTaskCheckInterval); - uint32 currentCount = taskTracker->connectPollCount; - if (currentCount >= maxCount) - { - ereport(WARNING, (errmsg("could not establish asynchronous " - "connection after %u ms", - NodeConnectionTimeout))); - - taskTracker->trackerStatus = TRACKER_CONNECTION_FAILED; - - MultiClientDisconnect(connectionId); - taskTracker->connectionId = INVALID_CONNECTION_ID; - } - } - - break; - } - - case TRACKER_CONNECTED: - case TRACKER_CONNECTION_FAILED: - { - /* if connected or failed to connect in previous pass, reset poll count */ - taskTracker->connectPollCount = 0; - break; - } - - default: - { - int trackerStatus = (int) taskTracker->trackerStatus; - ereport(FATAL, (errmsg("invalid task tracker status: %d", trackerStatus))); - break; - } - } - - return taskTracker->trackerStatus; -} - - -/* - * ResolveTaskTracker is a helper function that resolves the task tracker from - * the given task and task execution. The function first finds the worker node - * the given task is scheduled to, and resolves the corresponding task tracker. - */ -static TaskTracker * -ResolveTaskTracker(HTAB *trackerHash, Task *task, TaskExecution *taskExecution) -{ - List *taskPlacementList = task->taskPlacementList; - uint32 currentIndex = taskExecution->currentNodeIndex; - - ShardPlacement *taskPlacement = list_nth(taskPlacementList, currentIndex); - char *nodeName = taskPlacement->nodeName; - uint32 nodePort = taskPlacement->nodePort; - - /* look up in the tracker hash for the found node name/port */ - TaskTracker *taskTracker = TrackerHashLookup(trackerHash, nodeName, nodePort); - Assert(taskTracker != NULL); - - return taskTracker; -} - - -/* - * ResolveMapTaskTracker is a helper function that finds the downstream map task - * dependency from the given task, and then resolves the task tracker for this - * map task. - */ -static TaskTracker * -ResolveMapTaskTracker(HTAB *trackerHash, Task *task, TaskExecution *taskExecution) -{ - /* we only resolve source (map) task tracker for map output fetch tasks */ - if (task->taskType != MAP_OUTPUT_FETCH_TASK) - { - return NULL; - } - - Assert(task->dependentTaskList != NIL); - Task *mapTask = (Task *) linitial(task->dependentTaskList); - TaskExecution *mapTaskExecution = mapTask->taskExecution; - - TaskTracker *mapTaskTracker = ResolveTaskTracker(trackerHash, mapTask, - mapTaskExecution); - Assert(mapTaskTracker != NULL); - - return mapTaskTracker; -} - - -/* - * TrackerHashLookup looks for the task tracker that corresponds to the given - * node name and port number, and returns the found task tracker if any. - */ -static TaskTracker * -TrackerHashLookup(HTAB *trackerHash, const char *nodeName, uint32 nodePort) -{ - bool handleFound = false; - - TaskTracker taskTrackerKey; - memset(taskTrackerKey.workerName, 0, WORKER_LENGTH); - strlcpy(taskTrackerKey.workerName, nodeName, WORKER_LENGTH); - taskTrackerKey.workerPort = nodePort; - - void *hashKey = (void *) &taskTrackerKey; - TaskTracker *taskTracker = (TaskTracker *) hash_search(trackerHash, hashKey, - HASH_FIND, &handleFound); - if (taskTracker == NULL || !handleFound) - { - ereport(ERROR, (errmsg("could not find task tracker for node \"%s:%u\"", - nodeName, nodePort))); - } - - return taskTracker; -} - - -/* - * ManageTaskExecution manages all execution logic for the given task. For this, - * the function checks if the task's downstream dependencies have completed. If - * they have, the function assigns the task to the task tracker proxy object, - * and regularly checks the task's execution status. - * - * If the task completes, the function changes task's status. Else if the task - * observes a connection related failure, the function retries the task on the - * same task tracker. Else if the task tracker isn't considered as healthy, the - * function signals to the caller that the task needs to be assigned to another - * task tracker. - */ -static TaskExecStatus -ManageTaskExecution(TaskTracker *taskTracker, TaskTracker *sourceTaskTracker, - Task *task, TaskExecution *taskExecution) -{ - TaskExecStatus *taskStatusArray = taskExecution->taskStatusArray; - uint32 currentNodeIndex = taskExecution->currentNodeIndex; - - TaskExecStatus currentExecutionStatus = taskStatusArray[currentNodeIndex]; - TaskExecStatus nextExecutionStatus = EXEC_TASK_INVALID_FIRST; - - switch (currentExecutionStatus) - { - case EXEC_TASK_UNASSIGNED: - { - bool trackerHealthy = TrackerHealthy(taskTracker); - if (!trackerHealthy) - { - nextExecutionStatus = EXEC_TASK_TRACKER_FAILED; - break; - } - - /* - * We first retrieve this task's downstream dependencies, and then check - * if these dependencies' executions have completed. - */ - bool taskExecutionsCompleted = TaskExecutionsCompleted( - task->dependentTaskList); - if (!taskExecutionsCompleted) - { - nextExecutionStatus = EXEC_TASK_UNASSIGNED; - break; - } - - /* if map fetch task, create query string from completed map task */ - TaskType taskType = task->taskType; - if (taskType == MAP_OUTPUT_FETCH_TASK) - { - Task *mapTask = (Task *) linitial(task->dependentTaskList); - TaskExecution *mapTaskExecution = mapTask->taskExecution; - - StringInfo mapFetchTaskQueryString = MapFetchTaskQueryString(task, - mapTask); - SetTaskQueryString(task, mapFetchTaskQueryString->data); - taskExecution->querySourceNodeIndex = mapTaskExecution->currentNodeIndex; - } - - /* - * We finally queue this task for execution. Note that we queue sql and - * other tasks slightly differently. - */ - if (taskType == READ_TASK) - { - TrackerQueueSqlTask(taskTracker, task); - } - else - { - TrackerQueueTask(taskTracker, task); - } - - nextExecutionStatus = EXEC_TASK_QUEUED; - break; - } - - case EXEC_TASK_QUEUED: - { - bool trackerHealthy = TrackerHealthy(taskTracker); - if (!trackerHealthy) - { - nextExecutionStatus = EXEC_TASK_TRACKER_FAILED; - break; - } - - TaskStatus remoteTaskStatus = TrackerTaskStatus(taskTracker, task); - if (remoteTaskStatus == TASK_SUCCEEDED) - { - nextExecutionStatus = EXEC_TASK_DONE; - } - else if (remoteTaskStatus == TASK_CLIENT_SIDE_ASSIGN_FAILED || - remoteTaskStatus == TASK_CLIENT_SIDE_STATUS_FAILED) - { - nextExecutionStatus = EXEC_TASK_TRACKER_RETRY; - } - else if (remoteTaskStatus == TASK_PERMANENTLY_FAILED) - { - /* - * If a map output fetch task failed, we assume the problem lies with - * the map task (and the source task tracker it runs on). Otherwise, - * we assume the task tracker crashed, and fail over to the next task - * tracker. - */ - if (task->taskType == MAP_OUTPUT_FETCH_TASK) - { - nextExecutionStatus = EXEC_SOURCE_TASK_TRACKER_RETRY; - } - else - { - nextExecutionStatus = EXEC_TASK_TRACKER_FAILED; - } - } - else - { - /* assume task is still in progress */ - nextExecutionStatus = EXEC_TASK_QUEUED; - } - - break; - } - - case EXEC_TASK_TRACKER_RETRY: - { - /* - * This case statement usually handles connection related issues. Some - * edge cases however, like a user sending a SIGTERM to the worker node, - * keep the connection open but disallow task assignments. We therefore - * need to track those as intermittent tracker failures here. - */ - bool trackerConnectionUp = TrackerConnectionUp(taskTracker); - if (trackerConnectionUp) - { - taskTracker->trackerFailureCount++; - } - - bool trackerHealthy = TrackerHealthy(taskTracker); - if (trackerHealthy) - { - TaskStatus remoteTaskStatus = TrackerTaskStatus(taskTracker, task); - if (remoteTaskStatus == TASK_CLIENT_SIDE_ASSIGN_FAILED) - { - nextExecutionStatus = EXEC_TASK_UNASSIGNED; - } - else if (remoteTaskStatus == TASK_CLIENT_SIDE_STATUS_FAILED) - { - nextExecutionStatus = EXEC_TASK_QUEUED; - } - } - else - { - nextExecutionStatus = EXEC_TASK_TRACKER_FAILED; - } - - break; - } - - case EXEC_SOURCE_TASK_TRACKER_RETRY: - { - Task *mapTask = (Task *) linitial(task->dependentTaskList); - TaskExecution *mapTaskExecution = mapTask->taskExecution; - uint32 sourceNodeIndex = mapTaskExecution->currentNodeIndex; - - Assert(sourceTaskTracker != NULL); - Assert(task->taskType == MAP_OUTPUT_FETCH_TASK); - - /* - * As this map fetch task was running, another map fetch that depends on - * another map task might have failed. We would have then reassigned the - * map task and potentially other map tasks in its constraint group. So - * this map fetch's source node might have changed underneath us. If it - * did, we don't want to record a failure for the new source tracker. - */ - if (taskExecution->querySourceNodeIndex == sourceNodeIndex) - { - bool sourceTrackerConnectionUp = TrackerConnectionUp(sourceTaskTracker); - if (sourceTrackerConnectionUp) - { - sourceTaskTracker->trackerFailureCount++; - } - } - - bool sourceTrackerHealthy = TrackerHealthy(sourceTaskTracker); - if (sourceTrackerHealthy) - { - /* - * We change our status to unassigned. In that status, we queue an - * "update map fetch task" on the task tracker, and retry fetching - * the map task's output from the same source node. - */ - nextExecutionStatus = EXEC_TASK_UNASSIGNED; - } - else - { - nextExecutionStatus = EXEC_SOURCE_TASK_TRACKER_FAILED; - } - - break; - } - - case EXEC_TASK_TRACKER_FAILED: - case EXEC_SOURCE_TASK_TRACKER_FAILED: - { - /* - * These two cases exist to signal to the caller that we failed. In both - * cases, the caller is responsible for reassigning task(s) and running - * the appropriate recovery logic. - */ - nextExecutionStatus = EXEC_TASK_UNASSIGNED; - break; - } - - case EXEC_TASK_DONE: - { - /* we are done with this task's execution */ - nextExecutionStatus = EXEC_TASK_DONE; - break; - } - - default: - { - /* we fatal here to avoid leaking client-side resources */ - ereport(FATAL, (errmsg("invalid execution status: %d", - currentExecutionStatus))); - break; - } - } - - /* update task execution's status for most recent task tracker */ - uint32 nextNodeIndex = taskExecution->currentNodeIndex; - taskStatusArray[nextNodeIndex] = nextExecutionStatus; - - return nextExecutionStatus; -} - - -/* - * ManageTransmitExecution manages logic to fetch the results of the given SQL - * task to the master node. For this, the function checks if the given SQL task - * has completed. If it has, the function starts the copy out protocol to fetch - * the task's results and write them to the local filesystem. When the transmit - * completes or fails, the function notes that by changing the transmit status. - */ -static TransmitExecStatus -ManageTransmitExecution(TaskTracker *transmitTracker, - Task *task, TaskExecution *taskExecution, - DistributedExecutionStats *executionStats) -{ - int32 *fileDescriptorArray = taskExecution->fileDescriptorArray; - uint32 currentNodeIndex = taskExecution->currentNodeIndex; - - TransmitExecStatus *transmitStatusArray = taskExecution->transmitStatusArray; - TransmitExecStatus currentTransmitStatus = transmitStatusArray[currentNodeIndex]; - TransmitExecStatus nextTransmitStatus = EXEC_TRANSMIT_INVALID_FIRST; - Assert(task->taskType == READ_TASK); - - switch (currentTransmitStatus) - { - case EXEC_TRANSMIT_UNASSIGNED: - { - TaskExecStatus *taskStatusArray = taskExecution->taskStatusArray; - TaskExecStatus currentExecutionStatus = taskStatusArray[currentNodeIndex]; - - /* if top level task's in progress, nothing to do */ - if (currentExecutionStatus != EXEC_TASK_DONE) - { - nextTransmitStatus = EXEC_TRANSMIT_UNASSIGNED; - break; - } - - bool trackerHealthy = TrackerHealthy(transmitTracker); - if (!trackerHealthy) - { - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_FAILED; - break; - } - - TrackerQueueFileTransmit(transmitTracker, task); - nextTransmitStatus = EXEC_TRANSMIT_QUEUED; - break; - } - - case EXEC_TRANSMIT_QUEUED: - { - bool trackerHealthy = TrackerHealthy(transmitTracker); - if (!trackerHealthy) - { - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_FAILED; - break; - } - - TaskStatus taskStatus = TrackerTaskStatus(transmitTracker, task); - if (taskStatus == TASK_FILE_TRANSMIT_QUEUED) - { - /* remain in queued status until tracker assigns this task */ - nextTransmitStatus = EXEC_TRANSMIT_QUEUED; - break; - } - else if (taskStatus == TASK_CLIENT_SIDE_TRANSMIT_FAILED) - { - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_RETRY; - break; - } - - /* the open connection belongs to this task */ - int32 connectionId = TransmitTrackerConnectionId(transmitTracker, task); - Assert(connectionId != INVALID_CONNECTION_ID); - Assert(taskStatus == TASK_ASSIGNED); - - /* start copy protocol */ - QueryStatus queryStatus = MultiClientQueryStatus(connectionId); - if (queryStatus == CLIENT_QUERY_COPY) - { - StringInfo jobDirectoryName = MasterJobDirectoryName(task->jobId); - StringInfo taskFilename = TaskFilename(jobDirectoryName, task->taskId); - - char *filename = taskFilename->data; - int fileFlags = (O_APPEND | O_CREAT | O_RDWR | O_TRUNC | PG_BINARY); - int fileMode = (S_IRUSR | S_IWUSR); - - int32 fileDescriptor = BasicOpenFilePerm(filename, fileFlags, fileMode); - if (fileDescriptor >= 0) - { - /* - * All files inside the job directory get automatically cleaned - * up on transaction commit or abort. - */ - fileDescriptorArray[currentNodeIndex] = fileDescriptor; - nextTransmitStatus = EXEC_TRANSMIT_COPYING; - } - else - { - ereport(WARNING, (errcode_for_file_access(), - errmsg("could not open file \"%s\": %m", - filename))); - - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_RETRY; - } - } - else - { - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_RETRY; - } - - /* - * We use task tracker logic to manage file transmits as well, but that - * abstraction starts to leak after we drop into the copy protocol. To - * make our task tracker logic work, we need to "void" the tracker's - * connection if the transmit task failed in here. - */ - if (nextTransmitStatus == EXEC_TRANSMIT_TRACKER_RETRY) - { - transmitTracker->connectionBusy = false; - transmitTracker->connectionBusyOnTask = NULL; - } - - break; - } - - case EXEC_TRANSMIT_COPYING: - { - int32 fileDescriptor = fileDescriptorArray[currentNodeIndex]; - int closed = -1; - uint64 bytesReceived = 0; - - /* the open connection belongs to this task */ - int32 connectionId = TransmitTrackerConnectionId(transmitTracker, task); - Assert(connectionId != INVALID_CONNECTION_ID); - - CopyStatus copyStatus = MultiClientCopyData(connectionId, fileDescriptor, - &bytesReceived); - - if (SubPlanLevel > 0) - { - executionStats->totalIntermediateResultSize += bytesReceived; - } - - if (copyStatus == CLIENT_COPY_MORE) - { - /* worker node continues to send more data, keep reading */ - nextTransmitStatus = EXEC_TRANSMIT_COPYING; - break; - } - - /* we are done copying data */ - if (copyStatus == CLIENT_COPY_DONE) - { - closed = close(fileDescriptor); - fileDescriptorArray[currentNodeIndex] = -1; - - if (closed >= 0) - { - nextTransmitStatus = EXEC_TRANSMIT_DONE; - } - else - { - ereport(WARNING, (errcode_for_file_access(), - errmsg("could not close copied file: %m"))); - - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_RETRY; - } - } - else if (copyStatus == CLIENT_COPY_FAILED) - { - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_RETRY; - - closed = close(fileDescriptor); - fileDescriptorArray[currentNodeIndex] = -1; - - if (closed < 0) - { - ereport(WARNING, (errcode_for_file_access(), - errmsg("could not close copy file: %m"))); - } - } - - /* - * We use task tracker logic to manage file transmits as well, but that - * abstraction leaks after we drop into the copy protocol. To make it - * work, we reset transmit tracker's connection for next file transmit. - */ - transmitTracker->connectionBusy = false; - transmitTracker->connectionBusyOnTask = NULL; - - break; - } - - case EXEC_TRANSMIT_TRACKER_RETRY: - { - /* - * The task tracker proxy handles connection errors. On the off chance - * that our connection is still up and the transmit tracker misbehaved, - * we capture this as an intermittent tracker failure. - */ - bool trackerConnectionUp = TrackerConnectionUp(transmitTracker); - if (trackerConnectionUp) - { - transmitTracker->trackerFailureCount++; - } - - bool trackerHealthy = TrackerHealthy(transmitTracker); - if (trackerHealthy) - { - nextTransmitStatus = EXEC_TRANSMIT_UNASSIGNED; - } - else - { - nextTransmitStatus = EXEC_TRANSMIT_TRACKER_FAILED; - } - - break; - } - - case EXEC_TRANSMIT_TRACKER_FAILED: - { - /* - * This case exists to signal to the caller that we failed. The caller - * is now responsible for reassigning the transmit task (and downstream - * SQL task dependencies) and running the appropriate recovery logic. - */ - nextTransmitStatus = EXEC_TRANSMIT_UNASSIGNED; - break; - } - - case EXEC_TRANSMIT_DONE: - { - /* we are done with fetching task results to the master node */ - nextTransmitStatus = EXEC_TRANSMIT_DONE; - break; - } - - default: - { - /* we fatal here to avoid leaking client-side resources */ - ereport(FATAL, (errmsg("invalid transmit status: %d", - currentTransmitStatus))); - break; - } - } - - /* update file transmit status for most recent transmit tracker */ - uint32 nextNodeIndex = taskExecution->currentNodeIndex; - transmitStatusArray[nextNodeIndex] = nextTransmitStatus; - - return nextTransmitStatus; -} - - -/* - * TaskExecutionsCompleted checks if all task executions in the given task list - * have completed. If they have, the function returns true. Note that this - * function takes the list of tasks as an optimization over separately - * extracting a list of task executions, but it should only operate on task - * executions to preserve the abstraction. - */ -static bool -TaskExecutionsCompleted(List *taskList) -{ - bool taskExecutionsComplete = true; - - Task *task = NULL; - foreach_ptr(task, taskList) - { - TaskExecution *taskExecution = task->taskExecution; - uint32 nodeIndex = taskExecution->currentNodeIndex; - - TaskExecStatus taskStatus = taskExecution->taskStatusArray[nodeIndex]; - if (taskStatus != EXEC_TASK_DONE) - { - taskExecutionsComplete = false; - break; - } - } - - return taskExecutionsComplete; -} - - -/* - * MapFetchTaskQueryString constructs the map fetch query string from the given - * map output fetch task and its downstream map task dependency. The constructed - * query string allows fetching the map task's partitioned output file from the - * worker node it's created to the worker node that will execute the merge task. - */ -static StringInfo -MapFetchTaskQueryString(Task *mapFetchTask, Task *mapTask) -{ - uint32 partitionFileId = mapFetchTask->partitionId; - uint32 mergeTaskId = mapFetchTask->upstreamTaskId; - - /* find the node name/port for map task's execution */ - List *mapTaskPlacementList = mapTask->taskPlacementList; - TaskExecution *mapTaskExecution = mapTask->taskExecution; - uint32 currentIndex = mapTaskExecution->currentNodeIndex; - - ShardPlacement *mapTaskPlacement = list_nth(mapTaskPlacementList, currentIndex); - char *mapTaskNodeName = mapTaskPlacement->nodeName; - uint32 mapTaskNodePort = mapTaskPlacement->nodePort; - - Assert(mapFetchTask->taskType == MAP_OUTPUT_FETCH_TASK); - Assert(mapTask->taskType == MAP_TASK); - - StringInfo mapFetchQueryString = makeStringInfo(); - appendStringInfo(mapFetchQueryString, MAP_OUTPUT_FETCH_COMMAND, - mapTask->jobId, mapTask->taskId, partitionFileId, - mergeTaskId, /* fetch results to merge task */ - mapTaskNodeName, mapTaskNodePort); - - return mapFetchQueryString; -} - - -/* - * TrackerQueueSqlTask wraps a copy out command around the given task's query, - * creates a task assignment query from this copy out command, and then queues - * this assignment query in the given tracker's internal hash. The queued query - * will be assigned to the remote task tracker at a later time. - */ -static void -TrackerQueueSqlTask(TaskTracker *taskTracker, Task *task) -{ - HTAB *taskStateHash = taskTracker->taskStateHash; - - /* - * We first wrap the original query string in a worker_execute_sql_task - * call. This allows for the query's results to persist on the worker node - * after the query completes and for the executor to later fetch this - * persisted data using COPY ... (format 'transmit') - */ - - StringInfo sqlTaskQueryString = makeStringInfo(); - char *escapedTaskQueryString = quote_literal_cstr(TaskQueryString(task)); - - if (BinaryMasterCopyFormat) - { - appendStringInfo(sqlTaskQueryString, EXECUTE_SQL_TASK_TO_FILE_BINARY, - task->jobId, task->taskId, escapedTaskQueryString); - } - else - { - appendStringInfo(sqlTaskQueryString, EXECUTE_SQL_TASK_TO_FILE_TEXT, - task->jobId, task->taskId, escapedTaskQueryString); - } - - /* wrap a task assignment query outside the copy out query */ - StringInfo taskAssignmentQuery = TaskAssignmentQuery(task, sqlTaskQueryString->data); - - TrackerTaskState *taskState = TaskStateHashEnter(taskStateHash, task->jobId, - task->taskId); - taskState->status = TASK_CLIENT_SIDE_QUEUED; - taskState->taskAssignmentQuery = taskAssignmentQuery; -} - - -/* - * TrackerQueueTask creates a task assignment query from the given task's query - * string, and then queues this assignment query in the given tracker's internal - * hash. The queued query will be assigned to the remote task tracker at a later - * time. - */ -static void -TrackerQueueTask(TaskTracker *taskTracker, Task *task) -{ - HTAB *taskStateHash = taskTracker->taskStateHash; - - /* wrap a task assignment query outside the original query */ - StringInfo taskAssignmentQuery = - TaskAssignmentQuery(task, TaskQueryString(task)); - - TrackerTaskState *taskState = TaskStateHashEnter(taskStateHash, task->jobId, - task->taskId); - taskState->status = TASK_CLIENT_SIDE_QUEUED; - taskState->taskAssignmentQuery = taskAssignmentQuery; -} - - -/* - * TaskAssignmentQuery escapes the given query string with quotes, and wraps - * this escaped query string inside a task assignment command. This way, the - * query can be assigned to the remote task tracker. - */ -static StringInfo -TaskAssignmentQuery(Task *task, char *queryString) -{ - /* quote the original query as a string literal */ - char *escapedQueryString = quote_literal_cstr(queryString); - - StringInfo taskAssignmentQuery = makeStringInfo(); - appendStringInfo(taskAssignmentQuery, TASK_ASSIGNMENT_QUERY, - task->jobId, task->taskId, escapedQueryString); - - return taskAssignmentQuery; -} - - -/* - * TrackerTaskStatus returns the remote execution status of the given task. Note - * that the task must have already been queued with the task tracker for status - * checking to happen. - */ -static TaskStatus -TrackerTaskStatus(TaskTracker *taskTracker, Task *task) -{ - HTAB *taskStateHash = taskTracker->taskStateHash; - - TrackerTaskState *taskState = TrackerTaskStateHashLookup(taskStateHash, task); - if (taskState == NULL) - { - const char *nodeName = taskTracker->workerName; - uint32 nodePort = taskTracker->workerPort; - - ereport(ERROR, (errmsg("could not find task state for job " UINT64_FORMAT - " and task %u", task->jobId, task->taskId), - errdetail("Task tracker: \"%s:%u\"", nodeName, nodePort))); - } - - return taskState->status; -} - - -/* - * TrackerTaskStateHashLookup looks for the task state entry for the given task - * in the task tracker's state hash. The function then returns the found task - * state entry, if any. - */ -static TrackerTaskState * -TrackerTaskStateHashLookup(HTAB *taskStateHash, Task *task) -{ - bool handleFound = false; - - TrackerTaskState taskStateKey; - taskStateKey.jobId = task->jobId; - taskStateKey.taskId = task->taskId; - - void *hashKey = (void *) &taskStateKey; - TrackerTaskState *taskState = (TrackerTaskState *) hash_search(taskStateHash, hashKey, - HASH_FIND, - &handleFound); - - return taskState; -} - - -/* Checks if the given task tracker is considered as healthy. */ -static bool -TrackerHealthy(TaskTracker *taskTracker) -{ - bool trackerHealthy = false; - - if (taskTracker->trackerFailureCount < MAX_TRACKER_FAILURE_COUNT && - taskTracker->connectionFailureCount < MAX_TRACKER_FAILURE_COUNT) - { - trackerHealthy = true; - } - - return trackerHealthy; -} - - -/* - * TrackerQueueFileTransmit queues a file transmit request in the given task - * tracker's internal hash. The queued request will be served at a later time. - */ -static void -TrackerQueueFileTransmit(TaskTracker *transmitTracker, Task *task) -{ - HTAB *transmitStateHash = transmitTracker->taskStateHash; - - TrackerTaskState *transmitState = TaskStateHashEnter(transmitStateHash, task->jobId, - task->taskId); - transmitState->status = TASK_FILE_TRANSMIT_QUEUED; -} - - -/* - * TaskStateHashEnter creates a new task state entry in the given task state - * hash, and checks that the task entry has been properly created. - */ -static TrackerTaskState * -TaskStateHashEnter(HTAB *taskStateHash, uint64 jobId, uint32 taskId) -{ - bool handleFound = false; - - TrackerTaskState taskStateKey; - taskStateKey.jobId = jobId; - taskStateKey.taskId = taskId; - - void *hashKey = (void *) &taskStateKey; - TrackerTaskState *taskState = (TrackerTaskState *) hash_search(taskStateHash, hashKey, - HASH_ENTER, - &handleFound); - - /* if same task queued twice, we overwrite previous entry */ - if (handleFound) - { - ereport(DEBUG1, (errmsg("multiple task state entries for job " - UINT64_FORMAT " and task %u", jobId, taskId))); - } - - /* init task state object */ - taskState->status = TASK_STATUS_INVALID_FIRST; - taskState->taskAssignmentQuery = NULL; - - return taskState; -} - - -/* - * TransmitTrackerConnectionId checks if the given tracker is transmitting the - * given task's results to the master node. If it is, the function returns the - * connectionId used in transmitting task results. If not, the function returns - * an invalid connectionId. - */ -static int32 -TransmitTrackerConnectionId(TaskTracker *transmitTracker, Task *task) -{ - int32 connectionId = INVALID_CONNECTION_ID; - - TrackerTaskState *transmitState = transmitTracker->connectionBusyOnTask; - if (transmitState != NULL) - { - /* we are transmitting results for this particular task */ - if (transmitState->jobId == task->jobId && - transmitState->taskId == task->taskId) - { - connectionId = transmitTracker->connectionId; - } - } - - return connectionId; -} - - -/* - * ConstrainedTaskList finds the given task's constraint group within the given - * task and execution list. We define a constraint group as all tasks that need - * to be assigned (or reassigned) to the same task tracker for query execution - * to complete. At a high level, compute tasks are part of the same constraint - * group. Also, the transitive closure of tasks that have the same merge task - * dependency are part of one constraint group. - */ -static List * -ConstrainedTaskList(List *taskAndExecutionList, Task *task) -{ - List *constrainedTaskList = NIL; - - /* - * We first check if this task depends on any merge tasks. If it does *not*, - * the task's dependency list becomes our tiny constraint group. - */ - List *mergeTaskList = ConstrainedMergeTaskList(taskAndExecutionList, task); - if (mergeTaskList == NIL) - { - constrainedTaskList = ConstrainedNonMergeTaskList(taskAndExecutionList, task); - - return constrainedTaskList; - } - - /* we first add merge tasks and their dependencies to our constraint group */ - Task *mergeTask = NULL; - foreach_ptr(mergeTask, mergeTaskList) - { - List *dependentTaskList = mergeTask->dependentTaskList; - - constrainedTaskList = lappend(constrainedTaskList, mergeTask); - constrainedTaskList = TaskListConcatUnique(constrainedTaskList, - dependentTaskList); - } - - /* - * We now pick the first merge task as our constraining task, and walk over - * the task list looking for any tasks that depend on the constraining merge - * task. Note that finding a task's upstream dependencies necessitates that - * we walk over all the tasks. If we want to optimize this later on, we can - * precompute a task list that excludes map fetch tasks. - */ - Task *constrainingTask = (Task *) linitial(mergeTaskList); - - List *upstreamTaskList = UpstreamDependencyList(taskAndExecutionList, - constrainingTask); - Assert(upstreamTaskList != NIL); - - Task *upstreamTask = NULL; - foreach_ptr(upstreamTask, upstreamTaskList) - { - List *dependentTaskList = upstreamTask->dependentTaskList; - - /* - * We already added merge tasks to our constrained list. We therefore use - * concat unique to ensure they don't get appended for a second time. - */ - constrainedTaskList = TaskListAppendUnique(constrainedTaskList, upstreamTask); - constrainedTaskList = TaskListConcatUnique(constrainedTaskList, - dependentTaskList); - } - - return constrainedTaskList; -} - - -/* - * ConstrainedNonMergeTaskList finds the constraint group for the given task, - * assuming that the given task doesn't have any merge task dependencies. This - * constraint group includes compute task. - */ -static List * -ConstrainedNonMergeTaskList(List *taskAndExecutionList, Task *task) -{ - Task *upstreamTask = NULL; - List *dependentTaskList = NIL; - - TaskType taskType = task->taskType; - if (taskType == READ_TASK || taskType == MAP_TASK) - { - upstreamTask = task; - dependentTaskList = upstreamTask->dependentTaskList; - } - Assert(upstreamTask != NULL); - - List *constrainedTaskList = list_make1(upstreamTask); - constrainedTaskList = list_concat(constrainedTaskList, dependentTaskList); - - return constrainedTaskList; -} - - -/* - * UpstreamDependencyList looks for the given task's upstream task dependencies - * in the given task and execution list. For this, the function walks across all - * tasks in the task list. This walk is expensive due to the number of map fetch - * tasks involved; and this function should be called sparingly. - */ -static List * -UpstreamDependencyList(List *taskAndExecutionList, Task *searchedTask) -{ - List *upstreamTaskList = NIL; - - Task *upstreamTask = NULL; - foreach_ptr(upstreamTask, taskAndExecutionList) - { - List *dependentTaskList = upstreamTask->dependentTaskList; - - /* - * The given task and its upstream dependency cannot be of the same type. - * We perform this check as an optimization. This way, we can quickly - * skip over upstream map fetch tasks if we aren't looking for them. - */ - if (upstreamTask->taskType == searchedTask->taskType) - { - continue; - } - - /* - * We walk over the upstream task's dependency list, and check if any of - * them is the task we are looking for. - */ - Task *dependentTask = NULL; - foreach_ptr(dependentTask, dependentTaskList) - { - if (TasksEqual(dependentTask, searchedTask)) - { - upstreamTaskList = lappend(upstreamTaskList, upstreamTask); - } - } - } - - return upstreamTaskList; -} - - -/* - * ConstrainedMergeTaskList finds any merge task dependencies for the given task. - * Note that a given task may have zero, one, or two merge task dependencies. To - * resolve all dependencies, the function first looks at the task's type. Then, - * the function may need to find the task's parent, and resolve any merge task - * dependencies from that parent task. - */ -static List * -ConstrainedMergeTaskList(List *taskAndExecutionList, Task *task) -{ - List *constrainedMergeTaskList = NIL; - TaskType taskType = task->taskType; - - /* - * We find the list of constraining merge tasks for the given task. If the - * given task is a SQL or map task, we simply need to find its merge task - * dependencies -- if any. - */ - if (taskType == READ_TASK || taskType == MAP_TASK) - { - constrainedMergeTaskList = MergeTaskList(task->dependentTaskList); - } - else if (taskType == MAP_OUTPUT_FETCH_TASK) - { - List *taskList = UpstreamDependencyList(taskAndExecutionList, task); - Task *mergeTask = (Task *) linitial(taskList); - - /* - * Once we resolve the merge task, we use the exact same logic as below - * to find any other merge task in our constraint group. - */ - List *upstreamTaskList = UpstreamDependencyList(taskAndExecutionList, mergeTask); - Task *upstreamTask = (Task *) linitial(upstreamTaskList); - - constrainedMergeTaskList = MergeTaskList(upstreamTask->dependentTaskList); - } - else if (taskType == MERGE_TASK) - { - List *upstreamTaskList = UpstreamDependencyList(taskAndExecutionList, task); - - /* - * A merge task can have multiple SQL/map task parents. We now get only - * one of those parents. We then search if the parent depends on another - * merge task besides us. - */ - Assert(upstreamTaskList != NIL); - Task *upstreamTask = (Task *) linitial(upstreamTaskList); - - constrainedMergeTaskList = MergeTaskList(upstreamTask->dependentTaskList); - } - - return constrainedMergeTaskList; -} - - -/* - * MergeTaskList walks over the given task list, finds the merge tasks in the - * list, and returns the found tasks in a new list. - */ -static List * -MergeTaskList(List *taskList) -{ - List *mergeTaskList = NIL; - - Task *task = NULL; - foreach_ptr(task, taskList) - { - if (task->taskType == MERGE_TASK) - { - mergeTaskList = lappend(mergeTaskList, task); - } - } - - return mergeTaskList; -} - - -/* - * ReassignTaskList walks over all tasks in the given task list, and reassigns - * each task's execution and transmit to the next worker node. This ensures that - * all tasks within the same constraint group are failed over to the next node - * together. The function also increments each task's failure counter. - */ -static void -ReassignTaskList(List *taskList) -{ - List *completedTaskList = NIL; - - /* - * As an optimization, we first find the SQL tasks whose results we already - * fetched to the master node. We don't need to re-execute these SQL tasks. - */ - Task *task = NULL; - foreach_ptr(task, taskList) - { - TaskExecution *taskExecution = task->taskExecution; - - bool transmitCompleted = TransmitExecutionCompleted(taskExecution); - if ((task->taskType == READ_TASK) && transmitCompleted) - { - completedTaskList = lappend(completedTaskList, task); - } - } - - taskList = TaskListDifference(taskList, completedTaskList); - - foreach_ptr(task, taskList) - { - TaskExecution *taskExecution = task->taskExecution; - - uint32 currentNodeIndex = taskExecution->currentNodeIndex; - TaskExecStatus *taskStatusArray = taskExecution->taskStatusArray; - TransmitExecStatus *transmitStatusArray = taskExecution->transmitStatusArray; - - /* - * We reset current task statuses in case we fail on all other worker - * nodes and come back to this one. - */ - taskStatusArray[currentNodeIndex] = EXEC_TASK_UNASSIGNED; - transmitStatusArray[currentNodeIndex] = EXEC_TRANSMIT_UNASSIGNED; - - /* update node index to try next worker node */ - AdjustStateForFailure(taskExecution); - } -} - - -/* - * ReassignMapFetchTaskList walks over tasks in the given task list, and resets - * their task execution status. This ensures that all map output fetch tasks are - * retried after the node executing the map task has been failed over. - */ -static void -ReassignMapFetchTaskList(List *mapFetchTaskList) -{ - Task *mapFetchTask = NULL; - foreach_ptr(mapFetchTask, mapFetchTaskList) - { - TaskExecution *mapFetchTaskExecution = mapFetchTask->taskExecution; - - TaskExecStatus *taskStatusArray = mapFetchTaskExecution->taskStatusArray; - uint32 currentNodeIndex = mapFetchTaskExecution->currentNodeIndex; - - /* - * We reassign to same task tracker knowing that the source task tracker - * (that we failed to fetch map output from) has changed. - */ - taskStatusArray[currentNodeIndex] = EXEC_TASK_UNASSIGNED; - } -} - - -/* - * ManageTaskTracker manages tasks assigned to the given task tracker. For this, - * the function coordinates access to the underlying connection. The function - * also: (1) synchronously assigns locally queued tasks to the task tracker, (2) - * issues an asynchronous task status query for one assigned task at a time, and - * (3) retrieves status query results for the previously issued status query. - */ -static void -ManageTaskTracker(TaskTracker *taskTracker) -{ - bool trackerHealthy = TrackerHealthy(taskTracker); - if (!trackerHealthy) - { - return; - } - - bool trackerConnectionUp = TrackerConnectionUp(taskTracker); - if (!trackerConnectionUp) - { - TrackerReconnectPoll(taskTracker); /* try an async reconnect */ - return; - } - - /* - * (1) We first synchronously assign any pending new tasks. We also make - * sure not to block execution on one task tracker for a long time. - */ - if (!taskTracker->connectionBusy) - { - List *previousTaskList = taskTracker->assignedTaskList; - List *newTaskList = AssignQueuedTasks(taskTracker); - - taskTracker->assignedTaskList = list_concat(previousTaskList, newTaskList); - } - - /* - * (2) We find assigned tasks. We then send an asynchronous query to check - * the tasks' statuses. - */ - if (!taskTracker->connectionBusy) - { - List *taskStatusBatchList = TaskStatusBatchList(taskTracker); - - /* if we have running tasks, check their status */ - if (taskStatusBatchList) - { - int32 connectionId = taskTracker->connectionId; - - StringInfo taskStatusBatchQuery = TaskStatusBatchQuery(taskStatusBatchList); - - bool querySent = MultiClientSendQuery(connectionId, - taskStatusBatchQuery->data); - if (querySent) - { - taskTracker->connectionBusy = true; - taskTracker->connectionBusyOnTaskList = taskStatusBatchList; - } - else - { - /* mark only first task in list as failed */ - TrackerTaskState *taskState = (TrackerTaskState *) linitial( - taskStatusBatchList); - taskState->status = TASK_CLIENT_SIDE_STATUS_FAILED; - - list_free(taskStatusBatchList); - - taskTracker->connectionBusy = false; - taskTracker->connectionBusyOnTaskList = NIL; - } - - pfree(taskStatusBatchQuery); - } - } - - /* - * (3) check if results are ready for previously issued task status query - */ - if (taskTracker->connectionBusy) - { - int32 connectionId = taskTracker->connectionId; - - /* if connection is available, update task status accordingly */ - ResultStatus resultStatus = MultiClientResultStatus(connectionId); - if (resultStatus == CLIENT_RESULT_READY) - { - ReceiveTaskStatusBatchQueryResponse(taskTracker); - } - else if (resultStatus == CLIENT_RESULT_UNAVAILABLE) - { - TrackerTaskState *taskState = (TrackerTaskState *) linitial( - taskTracker->connectionBusyOnTaskList); - Assert(taskState != NULL); - taskState->status = TASK_CLIENT_SIDE_STATUS_FAILED; - } - - /* if connection is available, give it back to the task tracker */ - if (resultStatus != CLIENT_RESULT_BUSY) - { - list_free(taskTracker->connectionBusyOnTaskList); - - taskTracker->connectionBusy = false; - taskTracker->connectionBusyOnTaskList = NIL; - } - } -} - - -/* - * TrackerConnectionUp checks the most recent connection status for the given - * task tracker. The function returns true if the connection is still up. - */ -static bool -TrackerConnectionUp(TaskTracker *taskTracker) -{ - bool connectionUp = false; - - /* if we think we have a connection, check its most recent status */ - if (taskTracker->trackerStatus == TRACKER_CONNECTED) - { - connectionUp = MultiClientConnectionUp(taskTracker->connectionId); - } - - return connectionUp; -} - - -/* - * TrackerReconnectPoll checks if we have an open connection to the given task - * tracker. If not, the function opens an asynchronous connection to the task - * tracker and polls this connection's status on every call. The function also - * sets the task tracker's internal state. - */ -static void -TrackerReconnectPoll(TaskTracker *taskTracker) -{ - TrackerStatus currentStatus = taskTracker->trackerStatus; - if (currentStatus == TRACKER_CONNECTED) - { - bool connectionUp = MultiClientConnectionUp(taskTracker->connectionId); - if (connectionUp) - { - taskTracker->trackerStatus = TRACKER_CONNECTED; - } - else - { - taskTracker->trackerStatus = TRACKER_CONNECTION_FAILED; - - /* we lost the connection underneath us, clean it up */ - MultiClientDisconnect(taskTracker->connectionId); - taskTracker->connectionId = INVALID_CONNECTION_ID; - } - } - else if (currentStatus == TRACKER_CONNECT_START || - currentStatus == TRACKER_CONNECT_POLL) - { - taskTracker->trackerStatus = TrackerConnectPoll(taskTracker); - } - else if (currentStatus == TRACKER_CONNECTION_FAILED) - { - taskTracker->connectionFailureCount++; - taskTracker->connectPollCount = 0; - - taskTracker->trackerStatus = TRACKER_CONNECT_START; - } -} - - -/* - * AssignQueuedTasks walks over the given task tracker's task state hash, finds - * queued tasks in this hash, and synchronously assigns them to the given task - * tracker. The function then returns the list of newly assigned tasks. - */ -static List * -AssignQueuedTasks(TaskTracker *taskTracker) -{ - HTAB *taskStateHash = taskTracker->taskStateHash; - List *assignedTaskList = NIL; - uint32 taskAssignmentCount = 0; - List *tasksToAssignList = NIL; - StringInfo assignTaskBatchQuery = makeStringInfo(); - int32 connectionId = taskTracker->connectionId; - - HASH_SEQ_STATUS status; - hash_seq_init(&status, taskStateHash); - - TrackerTaskState *taskState = (TrackerTaskState *) hash_seq_search(&status); - while (taskState != NULL) - { - if (taskState->status == TASK_CLIENT_SIDE_QUEUED) - { - StringInfo taskAssignmentQuery = taskState->taskAssignmentQuery; - - appendStringInfo(assignTaskBatchQuery, "%s", taskAssignmentQuery->data); - - tasksToAssignList = lappend(tasksToAssignList, taskState); - taskAssignmentCount++; - if (taskAssignmentCount >= MaxAssignTaskBatchSize) - { - hash_seq_term(&status); - break; - } - } - - taskState = (TrackerTaskState *) hash_seq_search(&status); - } - - if (taskAssignmentCount > 0) - { - void *queryResult = NULL; - int rowCount = 0; - int columnCount = 0; - - bool batchSuccess = MultiClientSendQuery(connectionId, - assignTaskBatchQuery->data); - - foreach_ptr(taskState, tasksToAssignList) - { - if (!batchSuccess) - { - taskState->status = TASK_CLIENT_SIDE_ASSIGN_FAILED; - continue; - } - - BatchQueryStatus queryStatus = MultiClientBatchResult(connectionId, - &queryResult, - &rowCount, - &columnCount); - if (queryStatus == CLIENT_BATCH_QUERY_CONTINUE) - { - taskState->status = TASK_ASSIGNED; - assignedTaskList = lappend(assignedTaskList, taskState); - } - else - { - taskState->status = TASK_CLIENT_SIDE_ASSIGN_FAILED; - batchSuccess = false; - } - - MultiClientClearResult(queryResult); - } - - /* call MultiClientBatchResult one more time to finish reading results */ - MultiClientBatchResult(connectionId, &queryResult, &rowCount, &columnCount); - Assert(queryResult == NULL); - - pfree(assignTaskBatchQuery); - list_free(tasksToAssignList); - } - - return assignedTaskList; -} - - -/* - * TaskStatusBatchList returns a list containing up to MaxTaskStatusBatchSize - * tasks from the list of assigned tasks. When the number of tasks is greater - * than the maximum, the next call of this function will continue in the - * assigned task list after the last task that was added to the current list. - * - * In some cases the list may be empty even if tasks have been assigned due to - * wrap-around, namely if we first generate a batch of MaxTaskStatusBatchSize, - * but none of the remaining tasks in assignedTaskList are running. - */ -static List * -TaskStatusBatchList(TaskTracker *taskTracker) -{ - int32 assignedTaskIndex = 0; - List *assignedTaskList = taskTracker->assignedTaskList; - List *taskStatusBatchList = NIL; - - int32 assignedTaskCount = list_length(assignedTaskList); - if (assignedTaskCount == 0) - { - return NIL; - } - - int32 lastTaskIndex = (assignedTaskCount - 1); - int32 currentTaskIndex = taskTracker->currentTaskIndex; - if (currentTaskIndex >= lastTaskIndex) - { - currentTaskIndex = -1; - } - - TrackerTaskState *assignedTask = NULL; - foreach_ptr(assignedTask, assignedTaskList) - { - TaskStatus taskStatus = assignedTask->status; - - bool taskRunning = false; - if (taskStatus == TASK_ASSIGNED || taskStatus == TASK_SCHEDULED || - taskStatus == TASK_RUNNING || taskStatus == TASK_FAILED) - { - taskRunning = true; - } - - if (taskRunning && (assignedTaskIndex > currentTaskIndex)) - { - taskStatusBatchList = lappend(taskStatusBatchList, assignedTask); - if (list_length(taskStatusBatchList) >= MaxTaskStatusBatchSize) - { - break; - } - } - - assignedTaskIndex++; - } - - /* continue where we left off next time this function is called */ - taskTracker->currentTaskIndex = assignedTaskIndex; - - return taskStatusBatchList; -} - - -/* - * TaskStatusBatchQuery builds a command string containing multiple - * task_tracker_task_status queries from a TrackerTaskState list. - */ -static StringInfo -TaskStatusBatchQuery(List *taskList) -{ - StringInfo taskStatusBatchQuery = makeStringInfo(); - - TrackerTaskState *taskState = NULL; - foreach_ptr(taskState, taskList) - { - appendStringInfo(taskStatusBatchQuery, TASK_STATUS_QUERY, - taskState->jobId, taskState->taskId); - } - - return taskStatusBatchQuery; -} - - -/* - * ReceiveTaskStatusBatchQueryResponse assumes that a batch of task status - * queries have been previously sent to the given task tracker, and receives - * and processes the responses for these status queries. If a status check fails - * only one task status is marked as failed and the remainder is considered not - * executed. - */ -static void -ReceiveTaskStatusBatchQueryResponse(TaskTracker *taskTracker) -{ - List *checkedTaskList = taskTracker->connectionBusyOnTaskList; - int32 connectionId = taskTracker->connectionId; - int rowCount = 0; - int columnCount = 0; - void *queryResult = NULL; - - TrackerTaskState *checkedTask = NULL; - foreach_ptr(checkedTask, checkedTaskList) - { - TaskStatus taskStatus = TASK_STATUS_INVALID_FIRST; - - BatchQueryStatus queryStatus = MultiClientBatchResult(connectionId, &queryResult, - &rowCount, &columnCount); - if (queryStatus == CLIENT_BATCH_QUERY_CONTINUE) - { - char *valueString = MultiClientGetValue(queryResult, 0, 0); - if (valueString == NULL || (*valueString) == '\0') - { - taskStatus = TASK_PERMANENTLY_FAILED; - } - else - { - char *valueStringEnd = NULL; - errno = 0; - - taskStatus = strtoul(valueString, &valueStringEnd, 0); - if (errno != 0 || (*valueStringEnd) != '\0') - { - /* we couldn't parse received integer */ - taskStatus = TASK_PERMANENTLY_FAILED; - } - - Assert(taskStatus > TASK_STATUS_INVALID_FIRST); - Assert(taskStatus < TASK_STATUS_LAST); - } - } - else - { - taskStatus = TASK_CLIENT_SIDE_STATUS_FAILED; - } - - checkedTask->status = taskStatus; - - MultiClientClearResult(queryResult); - - if (queryStatus == CLIENT_BATCH_QUERY_FAILED) - { - /* remaining queries were not executed */ - break; - } - } - - /* call MultiClientBatchResult one more time to finish reading results */ - MultiClientBatchResult(connectionId, &queryResult, &rowCount, &columnCount); - Assert(queryResult == NULL); -} - - -/* - * ManageTransmitTracker manages access to the connection we opened to the worker - * node. If the connection is idle, and we have file transmit requests pending, - * the function picks a pending file transmit request, and starts the Copy Out - * protocol to copy the file's contents. - */ -static void -ManageTransmitTracker(TaskTracker *transmitTracker) -{ - bool trackerHealthy = TrackerHealthy(transmitTracker); - if (!trackerHealthy) - { - return; - } - - bool trackerConnectionUp = TrackerConnectionUp(transmitTracker); - if (!trackerConnectionUp) - { - TrackerReconnectPoll(transmitTracker); /* try an async reconnect */ - return; - } - - /* connection belongs to another file transmit */ - if (transmitTracker->connectionBusy) - { - return; - } - - TrackerTaskState *transmitState = NextQueuedFileTransmit( - transmitTracker->taskStateHash); - if (transmitState != NULL) - { - int32 connectionId = transmitTracker->connectionId; - StringInfo jobDirectoryName = JobDirectoryName(transmitState->jobId); - StringInfo taskFilename = TaskFilename(jobDirectoryName, transmitState->taskId); - char *userName = CurrentUserName(); - - StringInfo fileTransmitQuery = makeStringInfo(); - appendStringInfo(fileTransmitQuery, TRANSMIT_WITH_USER_COMMAND, - taskFilename->data, quote_literal_cstr(userName)); - - bool fileTransmitStarted = MultiClientSendQuery(connectionId, - fileTransmitQuery->data); - if (fileTransmitStarted) - { - transmitState->status = TASK_ASSIGNED; - - transmitTracker->connectionBusy = true; - transmitTracker->connectionBusyOnTask = transmitState; - } - else - { - transmitState->status = TASK_CLIENT_SIDE_TRANSMIT_FAILED; - - transmitTracker->connectionBusy = false; - transmitTracker->connectionBusyOnTask = NULL; - } - } -} - - -/* - * NextQueuedFileTransmit walks over all tasks in the given hash, and looks for - * a file transmit task that has been queued, but not served yet. - */ -static TrackerTaskState * -NextQueuedFileTransmit(HTAB *taskStateHash) -{ - HASH_SEQ_STATUS status; - hash_seq_init(&status, taskStateHash); - - TrackerTaskState *taskState = (TrackerTaskState *) hash_seq_search(&status); - while (taskState != NULL) - { - if (taskState->status == TASK_FILE_TRANSMIT_QUEUED) - { - hash_seq_term(&status); - break; - } - - taskState = (TrackerTaskState *) hash_seq_search(&status); - } - - return taskState; -} - - -/* - * JobIdList walks over all jobs in the given job tree and retrieves each job's - * identifier. The function then inserts these job identifiers in a new list and - * returns this list. - */ -static List * -JobIdList(Job *job) -{ - List *jobIdList = NIL; - - /* - * We walk over the job tree using breadth-first search. For this, we first - * queue the root node, and then start traversing our search space. - */ - List *jobQueue = list_make1(job); - while (jobQueue != NIL) - { - uint64 *jobIdPointer = (uint64 *) palloc0(sizeof(uint64)); - - Job *currJob = (Job *) linitial(jobQueue); - jobQueue = list_delete_first(jobQueue); - - (*jobIdPointer) = currJob->jobId; - jobIdList = lappend(jobIdList, jobIdPointer); - - /* prevent dependentJobList being modified on list_concat() call */ - List *jobChildrenList = list_copy(currJob->dependentJobList); - if (jobChildrenList != NIL) - { - jobQueue = list_concat(jobQueue, jobChildrenList); - } - } - - return jobIdList; -} - - -/* - * TrackerCleanupResources cleans up remote and local resources associated with - * the query. To clean up remote resources, the function cancels ongoing transmit - * tasks. It also waits for ongoing requests to the task trackers to complete - * before assigning "job clean up" tasks to them. To reclaim local resources, - * the function closes open file descriptors and disconnects from task trackers. - */ -static void -TrackerCleanupResources(HTAB *taskTrackerHash, HTAB *transmitTrackerHash, - List *jobIdList, List *taskList) -{ - /* - * We are done with query execution. We now wait for open requests to the task - * trackers to complete and cancel any open requests to the transmit trackers. - */ - TrackerHashWaitActiveRequest(taskTrackerHash); - TrackerHashCancelActiveRequest(transmitTrackerHash); - - /* only close open files; open connections are owned by trackers */ - Task *task = NULL; - foreach_ptr(task, taskList) - { - TaskExecution *taskExecution = task->taskExecution; - - CleanupTaskExecution(taskExecution); - task->taskExecution = NULL; - } - - /* - * For each executed job, we create a special task to clean up its resources - * on worker nodes, and send this clean-up task to all task trackers. - */ - uint64 *jobIdPointer = NULL; - foreach_ptr(jobIdPointer, jobIdList) - { - Task *jobCleanupTask = JobCleanupTask(*jobIdPointer); - TrackerHashCleanupJob(taskTrackerHash, jobCleanupTask); - } - - TrackerHashDisconnect(taskTrackerHash); - TrackerHashDisconnect(transmitTrackerHash); -} - - -/* - * TrackerHashWaitActiveRequest walks over task trackers in the given hash, and - * checks if they have an ongoing request. If they do, the function waits for - * the request to complete. If the request completes successfully, the function - * frees the connection for future tasks. - */ -static void -TrackerHashWaitActiveRequest(HTAB *taskTrackerHash) -{ - HASH_SEQ_STATUS status; - hash_seq_init(&status, taskTrackerHash); - - TaskTracker *taskTracker = (TaskTracker *) hash_seq_search(&status); - while (taskTracker != NULL) - { - bool trackerConnectionUp = TrackerConnectionUp(taskTracker); - - /* if we have an ongoing request, block until we have a response */ - if (trackerConnectionUp && taskTracker->connectionBusy) - { - QueryStatus queryStatus = MultiClientQueryStatus(taskTracker->connectionId); - if (queryStatus == CLIENT_QUERY_DONE) - { - taskTracker->connectionBusy = false; - taskTracker->connectionBusyOnTask = NULL; - taskTracker->connectionBusyOnTaskList = NIL; - } - } - - taskTracker = (TaskTracker *) hash_seq_search(&status); - } -} - - -/* - * TrackerHashCancelActiveRequest walks over task trackers in the given hash, - * and checks if they have an ongoing request. If they do, the function sends a - * cancel message on that connection. - */ -static void -TrackerHashCancelActiveRequest(HTAB *taskTrackerHash) -{ - HASH_SEQ_STATUS status; - hash_seq_init(&status, taskTrackerHash); - - TaskTracker *taskTracker = (TaskTracker *) hash_seq_search(&status); - while (taskTracker != NULL) - { - bool trackerConnectionUp = TrackerConnectionUp(taskTracker); - - /* if we have an ongoing request, send cancel message */ - if (trackerConnectionUp && taskTracker->connectionBusy) - { - MultiClientCancel(taskTracker->connectionId); - } - - taskTracker = (TaskTracker *) hash_seq_search(&status); - } -} - - -/* - * JobCleanupTask creates a special task to clean up all resources associated - * with a given job on the worker node. The function then returns this task. - */ -static Task * -JobCleanupTask(uint64 jobId) -{ - StringInfo jobCleanupQuery = makeStringInfo(); - appendStringInfo(jobCleanupQuery, JOB_CLEANUP_QUERY, jobId); - - Task *jobCleanupTask = CitusMakeNode(Task); - jobCleanupTask->jobId = jobId; - jobCleanupTask->taskId = JOB_CLEANUP_TASK_ID; - jobCleanupTask->replicationModel = REPLICATION_MODEL_INVALID; - SetTaskQueryString(jobCleanupTask, jobCleanupQuery->data); - - return jobCleanupTask; -} - - -/* - * TrackerHashCleanupJob walks over task trackers in the given hash, and assigns - * a job cleanup task to the tracker if the tracker's connection is available. - * The function then walks over task trackers to which it sent a cleanup task, - * checks the request's status, and emits an appropriate status message. - */ -static void -TrackerHashCleanupJob(HTAB *taskTrackerHash, Task *jobCleanupTask) -{ - uint64 jobId = jobCleanupTask->jobId; - List *taskTrackerList = NIL; - const long statusCheckInterval = 10000; /* microseconds */ - bool timedOut = false; - HASH_SEQ_STATUS status; - - hash_seq_init(&status, taskTrackerHash); - - /* walk over task trackers and try to issue job clean up requests */ - TaskTracker *taskTracker = (TaskTracker *) hash_seq_search(&status); - while (taskTracker != NULL) - { - bool trackerConnectionUp = TrackerConnectionUp(taskTracker); - if (trackerConnectionUp) - { - bool jobCleanupQuerySent = false; - - /* if we have a clear connection, send cleanup job */ - if (!taskTracker->connectionBusy) - { - /* assign through task tracker to manage resource utilization */ - StringInfo jobCleanupQuery = TaskAssignmentQuery( - jobCleanupTask, TaskQueryString(jobCleanupTask)); - - jobCleanupQuerySent = MultiClientSendQuery(taskTracker->connectionId, - jobCleanupQuery->data); - } - - /* - * If cleanup query was sent, mark that the connection is busy and - * hold onto the task tracker to check status. - */ - if (jobCleanupQuerySent) - { - taskTracker->connectionBusy = true; - taskTrackerList = lappend(taskTrackerList, taskTracker); - } - else - { - const char *nodeName = taskTracker->workerName; - uint32 nodePort = taskTracker->workerPort; - - ereport(WARNING, (errmsg("could not assign cleanup query for job " - UINT64_FORMAT " to node \"%s:%u\"", - jobId, nodeName, nodePort))); - } - } - - taskTracker = (TaskTracker *) hash_seq_search(&status); - } - - /* record the time when we start waiting for cleanup jobs to be sent */ - TimestampTz startTime = GetCurrentTimestamp(); - - /* - * Walk over task trackers to which we sent clean up requests. Perform - * these checks until it times out. - * - * We want to determine timedOut flag after the loop start to make sure - * we iterate one more time after time out occurs. This is necessary to report - * warning messages for timed out cleanup jobs. - */ - List *remainingTaskTrackerList = taskTrackerList; - while (list_length(remainingTaskTrackerList) > 0 && !timedOut) - { - List *activeTackTrackerList = remainingTaskTrackerList; - - remainingTaskTrackerList = NIL; - - pg_usleep(statusCheckInterval); - TimestampTz currentTime = GetCurrentTimestamp(); - timedOut = TimestampDifferenceExceeds(startTime, currentTime, - NodeConnectionTimeout); - - foreach_ptr(taskTracker, activeTackTrackerList) - { - int32 connectionId = taskTracker->connectionId; - char *nodeName = taskTracker->workerName; - uint32 nodePort = taskTracker->workerPort; - - ResultStatus resultStatus = MultiClientResultStatus(connectionId); - if (resultStatus == CLIENT_RESULT_READY) - { - QueryStatus queryStatus = MultiClientQueryStatus(connectionId); - if (queryStatus == CLIENT_QUERY_DONE) - { - ereport(DEBUG4, (errmsg("completed cleanup query for job " - UINT64_FORMAT, jobId))); - - /* clear connection for future cleanup queries */ - taskTracker->connectionBusy = false; - } - else if (timedOut) - { - ereport(WARNING, (errmsg("could not receive response for cleanup " - "query status for job " UINT64_FORMAT - " on node \"%s:%u\" with status %d", - jobId, - nodeName, nodePort, (int) queryStatus), - errhint("Manually clean job resources on node " - "\"%s:%u\" by running \"%s\" ", nodeName, - nodePort, TaskQueryString( - jobCleanupTask)))); - } - else - { - remainingTaskTrackerList = lappend(remainingTaskTrackerList, - taskTracker); - } - } - else if (resultStatus == CLIENT_RESULT_UNAVAILABLE || timedOut) - { - /* CLIENT_RESULT_UNAVAILABLE is returned if the connection failed somehow */ - ereport(WARNING, (errmsg("could not receive response for cleanup query " - "result for job " UINT64_FORMAT - " on node \"%s:%u\" with status %d", - jobId, nodeName, - nodePort, (int) resultStatus), - errhint("Manually clean job resources on node " - "\"%s:%u\" by running \"%s\" ", nodeName, - nodePort, TaskQueryString( - jobCleanupTask)))); - } - else - { - remainingTaskTrackerList = lappend(remainingTaskTrackerList, taskTracker); - } - } - } -} - - -/* - * TrackerHashDisconnect walks over task trackers in the given hash, and closes - * open connections to them. - */ -static void -TrackerHashDisconnect(HTAB *taskTrackerHash) -{ - HASH_SEQ_STATUS status; - hash_seq_init(&status, taskTrackerHash); - - TaskTracker *taskTracker = (TaskTracker *) hash_seq_search(&status); - while (taskTracker != NULL) - { - if (taskTracker->connectionId != INVALID_CONNECTION_ID) - { - MultiClientDisconnect(taskTracker->connectionId); - taskTracker->connectionId = INVALID_CONNECTION_ID; - } - - taskTracker = (TaskTracker *) hash_seq_search(&status); - } -} - - -/* - * TaskTrackerExecScan is a callback function which returns next tuple from a - * task-tracker execution. In the first call, it executes distributed task-tracker - * plan and loads results from temporary files into custom scan's tuple store. - * Then, it returns tuples one by one from this tuple store. - */ -TupleTableSlot * -TaskTrackerExecScan(CustomScanState *node) -{ - CitusScanState *scanState = (CitusScanState *) node; - - if (!scanState->finishedRemoteScan) - { - DistributedPlan *distributedPlan = scanState->distributedPlan; - Job *workerJob = distributedPlan->workerJob; - Query *jobQuery = workerJob->jobQuery; - - ErrorIfTransactionAccessedPlacementsLocally(); - DisableLocalExecution(); - - if (ContainsReadIntermediateResultFunction((Node *) jobQuery)) - { - ereport(ERROR, (errmsg("Complex subqueries and CTEs are not supported when " - "task_executor_type is set to 'task-tracker'"))); - } - - /* we are taking locks on partitions of partitioned tables */ - LockPartitionsInRelationList(distributedPlan->relationIdList, AccessShareLock); - - PrepareMasterJobDirectory(workerJob); - MultiTaskTrackerExecute(workerJob); - - LoadTuplesIntoTupleStore(scanState, workerJob); - - scanState->finishedRemoteScan = true; - } - - TupleTableSlot *resultSlot = ReturnTupleFromTuplestore(scanState); - - return resultSlot; -} - - -/* - * PrepareMasterJobDirectory creates a directory on the master node to keep job - * execution results. We also register this directory for automatic cleanup on - * portal delete. - */ -static void -PrepareMasterJobDirectory(Job *workerJob) -{ - StringInfo jobDirectoryName = MasterJobDirectoryName(workerJob->jobId); - CitusCreateDirectory(jobDirectoryName); - - ResourceOwnerEnlargeJobDirectories(CurrentResourceOwner); - ResourceOwnerRememberJobDirectory(CurrentResourceOwner, workerJob->jobId); -} diff --git a/src/backend/distributed/executor/query_stats.c b/src/backend/distributed/executor/query_stats.c index 04e6002c2..ee361a7c5 100644 --- a/src/backend/distributed/executor/query_stats.c +++ b/src/backend/distributed/executor/query_stats.c @@ -94,11 +94,6 @@ CitusExecutorName(MultiExecutorType executorType) return "adaptive"; } - case MULTI_EXECUTOR_TASK_TRACKER: - { - return "task-tracker"; - } - case MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT: { return "insert-select"; diff --git a/src/backend/distributed/executor/repartition_join_execution.c b/src/backend/distributed/executor/repartition_join_execution.c index 045f4b4f5..3a633d5c9 100644 --- a/src/backend/distributed/executor/repartition_join_execution.c +++ b/src/backend/distributed/executor/repartition_join_execution.c @@ -36,7 +36,7 @@ #include "distributed/metadata_cache.h" #include "distributed/multi_physical_planner.h" #include "distributed/multi_server_executor.h" -#include "distributed/multi_task_tracker_executor.h" +#include "distributed/task_execution_utils.h" #include "distributed/repartition_join_execution.h" #include "distributed/transaction_management.h" #include "distributed/transmit.h" @@ -63,8 +63,7 @@ ExecuteDependentTasks(List *topLevelTasks, Job *topLevelJob) { EnsureNoModificationsHaveBeenDone(); - bool createTaskExecution = false; - List *allTasks = CreateTaskListForJobTree(topLevelTasks, createTaskExecution); + List *allTasks = CreateTaskListForJobTree(topLevelTasks); EnsureCompatibleLocalExecutionState(allTasks); diff --git a/src/backend/distributed/planner/distributed_planner.c b/src/backend/distributed/planner/distributed_planner.c index 21daf030c..d2b3c4734 100644 --- a/src/backend/distributed/planner/distributed_planner.c +++ b/src/backend/distributed/planner/distributed_planner.c @@ -1276,6 +1276,9 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan) CustomScan *customScan = makeNode(CustomScan); MultiExecutorType executorType = MULTI_EXECUTOR_INVALID_FIRST; + /* this field is used in JobExecutorType */ + distributedPlan->relationIdList = localPlan->relationOids; + if (!distributedPlan->planningError) { executorType = JobExecutorType(distributedPlan); @@ -1289,12 +1292,6 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan) break; } - case MULTI_EXECUTOR_TASK_TRACKER: - { - customScan->methods = &TaskTrackerCustomScanMethods; - break; - } - case MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT: { customScan->methods = &NonPushableInsertSelectCustomScanMethods; @@ -1322,7 +1319,6 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan) } } - distributedPlan->relationIdList = localPlan->relationOids; distributedPlan->queryId = localPlan->queryId; Node *distributedPlanData = (Node *) distributedPlan; diff --git a/src/backend/distributed/planner/multi_logical_planner.c b/src/backend/distributed/planner/multi_logical_planner.c index ed792ba43..f2c046597 100644 --- a/src/backend/distributed/planner/multi_logical_planner.c +++ b/src/backend/distributed/planner/multi_logical_planner.c @@ -203,66 +203,6 @@ FindNodeCheck(Node *node, bool (*check)(Node *)) } -/* - * SingleRelationRepartitionSubquery returns true if it is eligible single - * repartition query planning in the sense that: - * - None of the levels of the subquery contains a join - * - Only a single RTE_RELATION exists, which means only a single table - * name is specified on the whole query - * - No sublinks exists in the subquery - * - No window functions exists in the subquery - * - * Note that the caller should still call DeferErrorIfUnsupportedSubqueryRepartition() - * to ensure that Citus supports the subquery. Also, this function is designed to run - * on the original query. - */ -bool -SingleRelationRepartitionSubquery(Query *queryTree) -{ - List *rangeTableIndexList = NULL; - List *rangeTableList = queryTree->rtable; - - /* we don't support subqueries in WHERE */ - if (queryTree->hasSubLinks) - { - return false; - } - - /* we don't support window functions */ - if (queryTree->hasWindowFuncs) - { - return false; - } - - /* - * Don't allow joins and set operations. If join appears in the queryTree, the - * length would be greater than 1. If only set operations exists, the length - * would be 0. - */ - ExtractRangeTableIndexWalker((Node *) queryTree->jointree, - &rangeTableIndexList); - if (list_length(rangeTableIndexList) != 1) - { - return false; - } - - int rangeTableIndex = linitial_int(rangeTableIndexList); - RangeTblEntry *rangeTableEntry = rt_fetch(rangeTableIndex, rangeTableList); - if (rangeTableEntry->rtekind == RTE_RELATION) - { - return true; - } - else if (rangeTableEntry->rtekind == RTE_SUBQUERY) - { - Query *subqueryTree = rangeTableEntry->subquery; - - return SingleRelationRepartitionSubquery(subqueryTree); - } - - return false; -} - - /* * TargetListOnPartitionColumn checks if at least one target list entry is on * partition column. diff --git a/src/backend/distributed/planner/multi_physical_planner.c b/src/backend/distributed/planner/multi_physical_planner.c index 7545267ba..6e5347b9a 100644 --- a/src/backend/distributed/planner/multi_physical_planner.c +++ b/src/backend/distributed/planner/multi_physical_planner.c @@ -52,7 +52,7 @@ #include "distributed/query_pushdown_planning.h" #include "distributed/shardinterval_utils.h" #include "distributed/shard_pruning.h" -#include "distributed/task_tracker.h" + #include "distributed/worker_manager.h" #include "distributed/worker_protocol.h" #include "distributed/version_compat.h" @@ -81,6 +81,8 @@ #include "utils/rel.h" #include "utils/typcache.h" +/* RepartitionJoinBucketCountPerNode determines bucket amount during repartitions */ +int RepartitionJoinBucketCountPerNode = 8; /* Policy to use when assigning tasks to worker nodes */ int TaskAssignmentPolicy = TASK_ASSIGNMENT_GREEDY; @@ -119,14 +121,12 @@ static MultiNode * LeftMostNode(MultiTreeRoot *multiTree); static Oid RangePartitionJoinBaseRelationId(MultiJoin *joinNode); static MultiTable * FindTableNode(MultiNode *multiNode, int rangeTableId); static Query * BuildJobQuery(MultiNode *multiNode, List *dependentJobList); -static Query * BuildReduceQuery(MultiExtendedOp *extendedOpNode, List *dependentJobList); static List * BaseRangeTableList(MultiNode *multiNode); static List * QueryTargetList(MultiNode *multiNode); static List * TargetEntryList(List *expressionList); static Node * AddAnyValueAggregates(Node *node, AddAnyValueAggregatesContext *context); static List * QueryGroupClauseList(MultiNode *multiNode); static List * QuerySelectClauseList(MultiNode *multiNode); -static List * QueryJoinClauseList(MultiNode *multiNode); static List * QueryFromList(List *rangeTableList); static Node * QueryJoinTree(MultiNode *multiNode, List *dependentJobList, List **rangeTableList); @@ -224,10 +224,6 @@ static List * MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex); static StringInfo ColumnNameArrayString(uint32 columnCount, uint64 generatingJobId); static StringInfo ColumnTypeArrayString(List *targetEntryList); -static StringInfo MergeTableQueryString(uint32 taskIdIndex, List *targetEntryList); -static StringInfo IntermediateTableQueryString(uint64 jobId, uint32 taskIdIndex, - Query *reduceQuery); -static uint32 FinalTargetEntryCount(List *targetEntryList); static bool CoPlacedShardIntervals(ShardInterval *firstInterval, ShardInterval *secondInterval); @@ -433,30 +429,6 @@ BuildJobTree(MultiTreeRoot *multiTree) loopDependentJobList = lappend(loopDependentJobList, mapMergeJob); } } - else if (boundaryNodeJobType == SUBQUERY_MAP_MERGE_JOB) - { - MultiPartition *partitionNode = (MultiPartition *) currentNode; - MultiNode *queryNode = GrandChildNode((MultiUnaryNode *) partitionNode); - Var *partitionKey = partitionNode->partitionColumn; - - /* build query and partition job */ - List *dependentJobList = list_copy(loopDependentJobList); - Query *jobQuery = BuildJobQuery(queryNode, dependentJobList); - - MapMergeJob *mapMergeJob = BuildMapMergeJob(jobQuery, dependentJobList, - partitionKey, - DUAL_HASH_PARTITION_TYPE, - InvalidOid, - SUBQUERY_MAP_MERGE_JOB); - - Query *reduceQuery = BuildReduceQuery((MultiExtendedOp *) parentNode, - list_make1(mapMergeJob)); - mapMergeJob->reduceQuery = reduceQuery; - - /* reset dependent job list */ - loopDependentJobList = NIL; - loopDependentJobList = list_make1(mapMergeJob); - } else if (boundaryNodeJobType == TOP_LEVEL_WORKER_JOB) { MultiNode *childNode = ChildNode((MultiUnaryNode *) currentNode); @@ -752,89 +724,6 @@ BuildJobQuery(MultiNode *multiNode, List *dependentJobList) } -/* - * BuildReduceQuery traverses the given logical plan tree, determines the job that - * corresponds to this part of the tree, and builds the query structure for that - * particular job. The function assumes that jobs this particular job depends on - * have already been built, as their output is needed to build the query. - */ -static Query * -BuildReduceQuery(MultiExtendedOp *extendedOpNode, List *dependentJobList) -{ - MultiNode *multiNode = (MultiNode *) extendedOpNode; - List *derivedRangeTableList = NIL; - List *targetList = NIL; - ListCell *columnCell = NULL; - List *columnNameList = NIL; - - Job *dependentJob = linitial(dependentJobList); - List *dependentTargetList = dependentJob->jobQuery->targetList; - uint32 columnCount = (uint32) list_length(dependentTargetList); - - for (uint32 columnIndex = 0; columnIndex < columnCount; columnIndex++) - { - StringInfo columnNameString = makeStringInfo(); - - appendStringInfo(columnNameString, MERGE_COLUMN_FORMAT, columnIndex); - - Value *columnValue = makeString(columnNameString->data); - columnNameList = lappend(columnNameList, columnValue); - } - - /* create a derived range table for the subtree below the collect */ - RangeTblEntry *rangeTableEntry = DerivedRangeTableEntry(multiNode, columnNameList, - OutputTableIdList(multiNode), - NIL, NIL, NIL, NIL); - rangeTableEntry->eref->colnames = columnNameList; - ModifyRangeTblExtraData(rangeTableEntry, CITUS_RTE_SHARD, NULL, NULL, NULL); - derivedRangeTableList = lappend(derivedRangeTableList, rangeTableEntry); - - targetList = copyObject(extendedOpNode->targetList); - List *columnList = pull_var_clause_default((Node *) targetList); - - foreach(columnCell, columnList) - { - Var *column = (Var *) lfirst(columnCell); - Index originalTableId = column->varnoold; - - /* find the new table identifier */ - Index newTableId = NewTableId(originalTableId, derivedRangeTableList); - column->varno = newTableId; - } - - /* build the where clause list using select and join predicates */ - List *selectClauseList = QuerySelectClauseList((MultiNode *) extendedOpNode); - List *joinClauseList = QueryJoinClauseList((MultiNode *) extendedOpNode); - List *whereClauseList = list_concat(selectClauseList, joinClauseList); - - /* - * Build the From/Where construct. We keep the where-clause list implicitly - * AND'd, since both partition and join pruning depends on the clauses being - * expressed as a list. - */ - FromExpr *joinTree = makeNode(FromExpr); - joinTree->quals = (Node *) whereClauseList; - joinTree->fromlist = QueryFromList(derivedRangeTableList); - - /* build the query structure for this job */ - Query *reduceQuery = makeNode(Query); - reduceQuery->commandType = CMD_SELECT; - reduceQuery->querySource = QSRC_ORIGINAL; - reduceQuery->canSetTag = true; - reduceQuery->rtable = derivedRangeTableList; - reduceQuery->targetList = targetList; - reduceQuery->jointree = joinTree; - reduceQuery->sortClause = extendedOpNode->sortClauseList; - reduceQuery->groupClause = extendedOpNode->groupClauseList; - reduceQuery->limitOffset = extendedOpNode->limitOffset; - reduceQuery->limitCount = extendedOpNode->limitCount; - reduceQuery->havingQual = extendedOpNode->havingQual; - reduceQuery->hasAggs = contain_aggs_of_level((Node *) targetList, 0); - - return reduceQuery; -} - - /* * BaseRangeTableList returns the list of range table entries for base tables in * the query. These base tables stand in contrast to derived tables generated by @@ -1201,44 +1090,6 @@ QuerySelectClauseList(MultiNode *multiNode) } -/* - * QueryJoinClauseList traverses the given logical plan tree, and extracts all - * join clauses from the join nodes. Note that this function does not walk below - * a collect node; the clauses below the collect node apply to another query, - * and they would have been captured by the remote job we depend upon. - */ -static List * -QueryJoinClauseList(MultiNode *multiNode) -{ - List *joinClauseList = NIL; - List *pendingNodeList = list_make1(multiNode); - - while (pendingNodeList != NIL) - { - MultiNode *currMultiNode = (MultiNode *) linitial(pendingNodeList); - CitusNodeTag nodeType = CitusNodeTag(currMultiNode); - pendingNodeList = list_delete_first(pendingNodeList); - - /* extract join clauses from the multi join node */ - if (nodeType == T_MultiJoin) - { - MultiJoin *joinNode = (MultiJoin *) currMultiNode; - List *clauseList = copyObject(joinNode->joinClauseList); - joinClauseList = list_concat(joinClauseList, clauseList); - } - - /* add this node's children only if the node isn't a multi collect */ - if (nodeType != T_MultiCollect) - { - List *childNodeList = ChildNodeList(currMultiNode); - pendingNodeList = list_concat(pendingNodeList, childNodeList); - } - } - - return joinClauseList; -} - - /* * Create a tree of JoinExpr and RangeTblRef nodes for the job query from * a given multiNode. If the tree contains MultiCollect or MultiJoin nodes, @@ -2108,7 +1959,7 @@ static uint32 HashPartitionCount(void) { uint32 groupCount = list_length(ActiveReadableNodeList()); - double maxReduceTasksPerNode = MaxRunningTasksPerNode / 2.0; + double maxReduceTasksPerNode = RepartitionJoinBucketCountPerNode; uint32 partitionCount = (uint32) rint(groupCount * maxReduceTasksPerNode); return partitionCount; @@ -4804,25 +4655,6 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex) mergeTask = CreateBasicTask(jobId, mergeTaskId, MERGE_TASK, mergeQueryString->data); } - else - { - StringInfo mergeTableQueryString = - MergeTableQueryString(taskIdIndex, targetEntryList); - char *escapedMergeTableQueryString = - quote_literal_cstr(mergeTableQueryString->data); - StringInfo intermediateTableQueryString = - IntermediateTableQueryString(jobId, taskIdIndex, reduceQuery); - char *escapedIntermediateTableQueryString = - quote_literal_cstr(intermediateTableQueryString->data); - StringInfo mergeAndRunQueryString = makeStringInfo(); - appendStringInfo(mergeAndRunQueryString, MERGE_FILES_AND_RUN_QUERY_COMMAND, - jobId, taskIdIndex, escapedMergeTableQueryString, - escapedIntermediateTableQueryString); - - mergeTask = CreateBasicTask(jobId, mergeTaskId, MERGE_TASK, - mergeAndRunQueryString->data); - } - mergeTask->partitionId = partitionId; taskIdIndex++; @@ -5842,129 +5674,3 @@ TaskListHighestTaskId(List *taskList) return highestTaskId; } - - -/* - * MergeTableQueryString builds a query string which creates a merge task table - * within the job's schema, which should have already been created by the task - * tracker protocol. - */ -static StringInfo -MergeTableQueryString(uint32 taskIdIndex, List *targetEntryList) -{ - StringInfo taskTableName = TaskTableName(taskIdIndex); - StringInfo mergeTableQueryString = makeStringInfo(); - StringInfo mergeTableName = makeStringInfo(); - StringInfo columnsString = makeStringInfo(); - ListCell *targetEntryCell = NULL; - uint32 columnIndex = 0; - - appendStringInfo(mergeTableName, "%s%s", taskTableName->data, MERGE_TABLE_SUFFIX); - - uint32 columnCount = (uint32) list_length(targetEntryList); - - foreach(targetEntryCell, targetEntryList) - { - TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); - Node *columnExpression = (Node *) targetEntry->expr; - Oid columnTypeId = exprType(columnExpression); - int32 columnTypeMod = exprTypmod(columnExpression); - - StringInfo columnNameString = makeStringInfo(); - appendStringInfo(columnNameString, MERGE_COLUMN_FORMAT, columnIndex); - - char *columnName = columnNameString->data; - char *columnType = format_type_with_typemod(columnTypeId, columnTypeMod); - - appendStringInfo(columnsString, "%s %s", columnName, columnType); - - columnIndex++; - if (columnIndex != columnCount) - { - appendStringInfo(columnsString, ", "); - } - } - - appendStringInfo(mergeTableQueryString, CREATE_TABLE_COMMAND, mergeTableName->data, - columnsString->data); - - return mergeTableQueryString; -} - - -/* - * IntermediateTableQueryString builds a query string which creates a task table - * by running reduce query on already created merge table. - */ -static StringInfo -IntermediateTableQueryString(uint64 jobId, uint32 taskIdIndex, Query *reduceQuery) -{ - StringInfo taskTableName = TaskTableName(taskIdIndex); - StringInfo intermediateTableQueryString = makeStringInfo(); - StringInfo mergeTableName = makeStringInfo(); - StringInfo columnsString = makeStringInfo(); - StringInfo taskReduceQueryString = makeStringInfo(); - Query *taskReduceQuery = copyObject(reduceQuery); - ListCell *columnNameCell = NULL; - uint32 columnIndex = 0; - - uint32 columnCount = FinalTargetEntryCount(reduceQuery->targetList); - List *columnNames = DerivedColumnNameList(columnCount, jobId); - - foreach(columnNameCell, columnNames) - { - Value *columnNameValue = (Value *) lfirst(columnNameCell); - char *columnName = strVal(columnNameValue); - - appendStringInfo(columnsString, "%s", columnName); - - columnIndex++; - if (columnIndex != columnCount) - { - appendStringInfo(columnsString, ", "); - } - } - - appendStringInfo(mergeTableName, "%s%s", taskTableName->data, MERGE_TABLE_SUFFIX); - - List *rangeTableList = taskReduceQuery->rtable; - RangeTblEntry *rangeTableEntry = (RangeTblEntry *) linitial(rangeTableList); - Alias *referenceNames = rangeTableEntry->eref; - referenceNames->aliasname = mergeTableName->data; - - rangeTableEntry->alias = rangeTableEntry->eref; - - ModifyRangeTblExtraData(rangeTableEntry, GetRangeTblKind(rangeTableEntry), - NULL, mergeTableName->data, NIL); - - pg_get_query_def(taskReduceQuery, taskReduceQueryString); - - appendStringInfo(intermediateTableQueryString, CREATE_TABLE_AS_COMMAND, - taskTableName->data, columnsString->data, - taskReduceQueryString->data); - - return intermediateTableQueryString; -} - - -/* - * FinalTargetEntryCount returns count of target entries in the final target - * entry list. - */ -static uint32 -FinalTargetEntryCount(List *targetEntryList) -{ - uint32 finalTargetEntryCount = 0; - ListCell *targetEntryCell = NULL; - - foreach(targetEntryCell, targetEntryList) - { - TargetEntry *targetEntry = (TargetEntry *) lfirst(targetEntryCell); - if (!targetEntry->resjunk) - { - finalTargetEntryCount++; - } - } - - return finalTargetEntryCount; -} diff --git a/src/backend/distributed/planner/multi_router_planner.c b/src/backend/distributed/planner/multi_router_planner.c index a912f442b..4913022e1 100644 --- a/src/backend/distributed/planner/multi_router_planner.c +++ b/src/backend/distributed/planner/multi_router_planner.c @@ -1590,7 +1590,6 @@ CreateTask(TaskType taskType) task->upstreamTaskId = INVALID_TASK_ID; task->shardInterval = NULL; task->assignmentConstrained = false; - task->taskExecution = NULL; task->replicationModel = REPLICATION_MODEL_INVALID; task->relationRowLockList = NIL; diff --git a/src/backend/distributed/planner/query_pushdown_planning.c b/src/backend/distributed/planner/query_pushdown_planning.c index 1e4b9549b..4296d9272 100644 --- a/src/backend/distributed/planner/query_pushdown_planning.c +++ b/src/backend/distributed/planner/query_pushdown_planning.c @@ -522,16 +522,7 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree, } else if (subqueryPushdownError) { - /* - * If not eligible for single relation repartition query, we should raise - * subquery pushdown error. - */ - bool singleRelationRepartitionSubquery = - SingleRelationRepartitionSubquery(originalQuery); - if (!singleRelationRepartitionSubquery) - { - RaiseDeferredErrorInternal(subqueryPushdownError, ERROR); - } + RaiseDeferredErrorInternal(subqueryPushdownError, ERROR); List *subqueryEntryList = SubqueryEntryList(queryTree); RangeTblEntry *subqueryRangeTableEntry = (RangeTblEntry *) linitial( diff --git a/src/backend/distributed/planner/recursive_planning.c b/src/backend/distributed/planner/recursive_planning.c index c65a860cd..cd0ab3c28 100644 --- a/src/backend/distributed/planner/recursive_planning.c +++ b/src/backend/distributed/planner/recursive_planning.c @@ -916,15 +916,6 @@ ShouldRecursivelyPlanSubquery(Query *subquery, RecursivePlanningContext *context */ return false; } - else if (TaskExecutorType == MULTI_EXECUTOR_TASK_TRACKER && - SingleRelationRepartitionSubquery(subquery)) - { - /* - * Citus can plan this and execute via repartitioning. Thus, - * no need to recursively plan. - */ - return false; - } return true; } diff --git a/src/backend/distributed/shared_library_init.c b/src/backend/distributed/shared_library_init.c index 706795b00..2470d154d 100644 --- a/src/backend/distributed/shared_library_init.c +++ b/src/backend/distributed/shared_library_init.c @@ -43,6 +43,7 @@ #include "distributed/coordinator_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/metadata_sync.h" +#include "distributed/multi_physical_planner.h" #include "distributed/multi_executor.h" #include "distributed/multi_explain.h" #include "distributed/multi_join_order.h" @@ -64,7 +65,7 @@ #include "distributed/shared_library_init.h" #include "distributed/statistics_collection.h" #include "distributed/subplan_execution.h" -#include "distributed/task_tracker.h" + #include "distributed/transaction_management.h" #include "distributed/transaction_recovery.h" #include "distributed/worker_log_messages.h" @@ -91,6 +92,7 @@ static char *CitusVersion = CITUS_VERSION; void _PG_init(void); +static void DoInitialCleanup(void); static void ResizeStackToMaximumDepth(void); static void multi_log_hook(ErrorData *edata); static void RegisterConnectionCleanup(void); @@ -107,9 +109,9 @@ static bool StatisticsCollectionGucCheckHook(bool *newval, void **extra, GucSour source); /* static variable to hold value of deprecated GUC variable */ -static bool ExpireCachedShards = false; -static int LargeTableShardCount = 0; -static int CitusSSLMode = 0; +static bool DeprecatedBool = false; +static int DeprecatedInt = 0; + /* *INDENT-OFF* */ /* GUC enum definitions */ @@ -136,7 +138,7 @@ static const struct config_enum_entry replication_model_options[] = { static const struct config_enum_entry task_executor_type_options[] = { { "adaptive", MULTI_EXECUTOR_ADAPTIVE, false }, { "real-time", DUMMY_REAL_TIME_EXECUTOR_ENUM_VALUE, false }, /* keep it for backward comp. */ - { "task-tracker", MULTI_EXECUTOR_TASK_TRACKER, false }, + { "task-tracker", MULTI_EXECUTOR_ADAPTIVE, false }, { NULL, 0, false } }; @@ -275,9 +277,6 @@ _PG_init(void) InitializeMaintenanceDaemon(); - /* organize that task tracker is started once server is up */ - TaskTrackerRegister(); - /* initialize coordinated transaction management */ InitializeTransactionManagement(); InitializeBackendManagement(); @@ -292,6 +291,20 @@ _PG_init(void) SetConfigOption("allow_system_table_mods", "true", PGC_POSTMASTER, PGC_S_OVERRIDE); } + + DoInitialCleanup(); +} + + +/* + * DoInitialCleanup does cleanup at start time. + * Currently it: + * - Removes repartition directories ( in case there are any leftovers) + */ +static void +DoInitialCleanup(void) +{ + RepartitionCleanupJobDirectories(); } @@ -464,7 +477,7 @@ RegisterCitusConfigVariables(void) gettext_noop("This variable has been deprecated. Use the citus.node_conninfo " "GUC instead."), NULL, - &CitusSSLMode, + &DeprecatedInt, 0, 0, 32, PGC_POSTMASTER, GUC_SUPERUSER_ONLY | GUC_NO_SHOW_ALL, @@ -472,13 +485,12 @@ RegisterCitusConfigVariables(void) DefineCustomBoolVariable( "citus.binary_master_copy_format", - gettext_noop("Use the binary master copy format."), - gettext_noop("When enabled, data is copied from workers to the master " - "in PostgreSQL's binary serialization format."), - &BinaryMasterCopyFormat, + gettext_noop("This GUC variable has been deprecated."), + NULL, + &DeprecatedBool, false, PGC_USERSET, - GUC_STANDARD, + GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomBoolVariable( @@ -497,10 +509,10 @@ RegisterCitusConfigVariables(void) "citus.expire_cached_shards", gettext_noop("This GUC variable has been deprecated."), NULL, - &ExpireCachedShards, + &DeprecatedBool, false, PGC_SIGHUP, - GUC_STANDARD, + GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomBoolVariable( @@ -1043,16 +1055,12 @@ RegisterCitusConfigVariables(void) DefineCustomIntVariable( "citus.task_tracker_delay", - gettext_noop("Task tracker sleep time between task management rounds."), - gettext_noop("The task tracker process wakes up regularly, walks over " - "all tasks assigned to it, and schedules and executes these " - "tasks. Then, the task tracker sleeps for a time period " - "before walking over these tasks again. This configuration " - "value determines the length of that sleeping period."), - &TaskTrackerDelay, + gettext_noop("This GUC variable has been deprecated."), + NULL, + &DeprecatedInt, 200 * MS, 1, 100 * MS_PER_SECOND, PGC_SIGHUP, - GUC_UNIT_MS | GUC_STANDARD, + GUC_UNIT_MS | GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomIntVariable( @@ -1071,43 +1079,43 @@ RegisterCitusConfigVariables(void) DefineCustomIntVariable( "citus.max_assign_task_batch_size", - gettext_noop("Sets the maximum number of tasks to assign per round."), - gettext_noop("The master node synchronously assigns tasks to workers in " - "batches. Bigger batches allow for faster task assignment, " - "but it may take longer for all workers to get tasks " - "if the number of workers is large. This configuration " - "value controls the maximum batch size."), - &MaxAssignTaskBatchSize, + gettext_noop("This GUC variable has been deprecated."), + NULL, + &DeprecatedInt, 64, 1, INT_MAX, PGC_USERSET, - GUC_STANDARD, + GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomIntVariable( "citus.max_tracked_tasks_per_node", - gettext_noop("Sets the maximum number of tracked tasks per node."), - gettext_noop("The task tracker processes keeps all assigned tasks in " - "a shared hash table, and schedules and executes these " - "tasks as appropriate. This configuration value limits " - "the size of the hash table, and therefore the maximum " - "number of tasks that can be tracked at any given time."), - &MaxTrackedTasksPerNode, + gettext_noop("This GUC variable has been deprecated."), + NULL, + &DeprecatedInt, 1024, 8, INT_MAX, PGC_POSTMASTER, - GUC_STANDARD, + GUC_STANDARD | GUC_NO_SHOW_ALL, + NULL, NULL, NULL); + + DefineCustomIntVariable( + "citus.repartition_join_bucket_count_per_node", + gettext_noop("Sets the bucket size for repartition joins per node"), + gettext_noop("Repartition joins create buckets in each node and " + "uses those to shuffle data around nodes. "), + &RepartitionJoinBucketCountPerNode, + 4, 1, INT_MAX, + PGC_SIGHUP, + GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomIntVariable( "citus.max_running_tasks_per_node", - gettext_noop("Sets the maximum number of tasks to run concurrently per node."), - gettext_noop("The task tracker process schedules and executes the tasks " - "assigned to it as appropriate. This configuration value " - "sets the maximum number of tasks to execute concurrently " - "on one node at any given time."), - &MaxRunningTasksPerNode, + gettext_noop("This GUC variable has been deprecated."), + NULL, + &DeprecatedInt, 8, 1, INT_MAX, PGC_SIGHUP, - GUC_STANDARD, + GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomIntVariable( @@ -1128,7 +1136,7 @@ RegisterCitusConfigVariables(void) "citus.large_table_shard_count", gettext_noop("This variable has been deprecated."), gettext_noop("Consider reference tables instead"), - &LargeTableShardCount, + &DeprecatedInt, 4, 1, 10000, PGC_USERSET, GUC_NO_SHOW_ALL, @@ -1392,15 +1400,12 @@ RegisterCitusConfigVariables(void) DefineCustomIntVariable( "citus.max_task_string_size", - gettext_noop("Sets the maximum size (in bytes) of a worker task call string."), - gettext_noop("Active worker tasks' are tracked in a shared hash table " - "on the master node. This configuration value limits the " - "maximum size of an individual worker task, and " - "affects the size of pre-allocated shared memory."), - &MaxTaskStringSize, + gettext_noop("This GUC variable has been deprecated."), + NULL, + &DeprecatedInt, 12288, 8192, 65536, PGC_POSTMASTER, - GUC_STANDARD, + GUC_STANDARD | GUC_NO_SHOW_ALL, NULL, NULL, NULL); DefineCustomBoolVariable( diff --git a/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql b/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql index 81e8cec8d..be2dd4c99 100644 --- a/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql +++ b/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql @@ -1,4 +1,15 @@ -- citus--9.4-1--9.5-1 -- bump version to 9.5-1 +SET search_path = 'pg_catalog'; +DROP FUNCTION task_tracker_assign_task(bigint, integer, text); +DROP FUNCTION task_tracker_task_status(bigint, integer); +DROP FUNCTION task_tracker_cleanup_job(bigint); +DROP FUNCTION worker_merge_files_and_run_query(bigint, integer, text, text); +DROP FUNCTION worker_execute_sql_task(bigint, integer, text, bool); +DROP TRIGGER dist_authinfo_task_tracker_cache_invalidate ON pg_catalog.pg_dist_authinfo; +DROP TRIGGER dist_poolinfo_task_tracker_cache_invalidate ON pg_catalog.pg_dist_poolinfo; +DROP FUNCTION task_tracker_conninfo_cache_invalidate(); + +RESET search_path; diff --git a/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql b/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql index 89f15a822..d4278110b 100644 --- a/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql +++ b/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql @@ -1,2 +1,59 @@ -- citus--9.5-1--9.4-1 --- this is an empty downgrade path since citus--9.4-1--9.5-1.sql is empty for now + +SET search_path = 'pg_catalog'; + +-- task_tracker_* functions + +CREATE FUNCTION task_tracker_assign_task(bigint, integer, text) + RETURNS void + LANGUAGE C STRICT + AS 'MODULE_PATHNAME', $$task_tracker_assign_task$$; +COMMENT ON FUNCTION task_tracker_assign_task(bigint, integer, text) + IS 'assign a task to execute'; + +CREATE FUNCTION task_tracker_task_status(bigint, integer) + RETURNS integer + LANGUAGE C STRICT + AS 'MODULE_PATHNAME', $$task_tracker_task_status$$; +COMMENT ON FUNCTION task_tracker_task_status(bigint, integer) + IS 'check an assigned task''s execution status'; + +CREATE FUNCTION task_tracker_cleanup_job(bigint) + RETURNS void + LANGUAGE C STRICT + AS 'MODULE_PATHNAME', $$task_tracker_cleanup_job$$; +COMMENT ON FUNCTION task_tracker_cleanup_job(bigint) + IS 'clean up all tasks associated with a job'; + +CREATE FUNCTION worker_merge_files_and_run_query(bigint, integer, text, text) + RETURNS void + LANGUAGE C STRICT + AS 'MODULE_PATHNAME', $$worker_merge_files_and_run_query$$; +COMMENT ON FUNCTION worker_merge_files_and_run_query(bigint, integer, text, text) + IS 'merge files and run a reduce query on merged files'; + +CREATE FUNCTION worker_execute_sql_task(jobid bigint, taskid integer, query text, binary bool) + RETURNS bigint + LANGUAGE C STRICT + AS 'MODULE_PATHNAME', $$worker_execute_sql_task$$; +COMMENT ON FUNCTION worker_execute_sql_task(bigint, integer, text, bool) + IS 'execute a query and write the results to a task file'; + +CREATE FUNCTION task_tracker_conninfo_cache_invalidate() + RETURNS trigger + LANGUAGE C + AS 'citus', $$task_tracker_conninfo_cache_invalidate$$; +COMMENT ON FUNCTION task_tracker_conninfo_cache_invalidate() + IS 'invalidate task-tracker conninfo cache'; + +CREATE TRIGGER dist_poolinfo_task_tracker_cache_invalidate + AFTER INSERT OR UPDATE OR DELETE OR TRUNCATE + ON pg_catalog.pg_dist_poolinfo + FOR EACH STATEMENT EXECUTE PROCEDURE task_tracker_conninfo_cache_invalidate(); + +CREATE TRIGGER dist_authinfo_task_tracker_cache_invalidate + AFTER INSERT OR UPDATE OR DELETE OR TRUNCATE + ON pg_catalog.pg_dist_authinfo + FOR EACH STATEMENT EXECUTE PROCEDURE task_tracker_conninfo_cache_invalidate(); + +RESET search_path; diff --git a/src/backend/distributed/test/distributed_intermediate_results.c b/src/backend/distributed/test/distributed_intermediate_results.c index 2f0f7994c..7a095504a 100644 --- a/src/backend/distributed/test/distributed_intermediate_results.c +++ b/src/backend/distributed/test/distributed_intermediate_results.c @@ -70,8 +70,13 @@ partition_task_list_results(PG_FUNCTION_ARGS) * Here SELECT query's target list should match column list of target relation, * so their partition column indexes are equal. */ - int partitionColumnIndex = targetRelation->partitionMethod != DISTRIBUTE_BY_NONE ? - targetRelation->partitionColumn->varattno - 1 : 0; + int partitionColumnIndex = 0; + + if (targetRelation->partitionMethod != DISTRIBUTE_BY_NONE && IsA( + targetRelation->partitionColumn, Var)) + { + partitionColumnIndex = targetRelation->partitionColumn->varattno - 1; + } List *fragmentList = PartitionTasklistResults(resultIdPrefix, taskList, partitionColumnIndex, diff --git a/src/backend/distributed/test/run_from_same_connection.c b/src/backend/distributed/test/run_from_same_connection.c index 06916461a..1e61ffc77 100644 --- a/src/backend/distributed/test/run_from_same_connection.c +++ b/src/backend/distributed/test/run_from_same_connection.c @@ -18,12 +18,13 @@ #include "access/xact.h" #include "distributed/connection_management.h" #include "distributed/function_utils.h" +#include "distributed/intermediate_result_pruning.h" #include "distributed/lock_graph.h" #include "distributed/coordinator_protocol.h" #include "distributed/metadata_cache.h" #include "distributed/remote_commands.h" #include "distributed/run_from_same_connection.h" -#include "distributed/task_tracker.h" + #include "distributed/version_compat.h" #include "executor/spi.h" #include "lib/stringinfo.h" diff --git a/src/backend/distributed/utils/citus_copyfuncs.c b/src/backend/distributed/utils/citus_copyfuncs.c index e676dabe8..09c96d2cf 100644 --- a/src/backend/distributed/utils/citus_copyfuncs.c +++ b/src/backend/distributed/utils/citus_copyfuncs.c @@ -315,7 +315,6 @@ CopyNodeTask(COPYFUNC_ARGS) COPY_SCALAR_FIELD(upstreamTaskId); COPY_NODE_FIELD(shardInterval); COPY_SCALAR_FIELD(assignmentConstrained); - COPY_NODE_FIELD(taskExecution); COPY_SCALAR_FIELD(replicationModel); COPY_SCALAR_FIELD(modifyWithSubquery); COPY_NODE_FIELD(relationShardList); @@ -342,26 +341,6 @@ CopyNodeLocalPlannedStatement(COPYFUNC_ARGS) } -void -CopyNodeTaskExecution(COPYFUNC_ARGS) -{ - DECLARE_FROM_AND_NEW_NODE(TaskExecution); - - COPY_SCALAR_FIELD(jobId); - COPY_SCALAR_FIELD(taskId); - COPY_SCALAR_FIELD(nodeCount); - - COPY_SCALAR_ARRAY(taskStatusArray, TaskExecStatus, from->nodeCount); - COPY_SCALAR_ARRAY(transmitStatusArray, TransmitExecStatus, from->nodeCount); - COPY_SCALAR_ARRAY(connectionIdArray, int32, from->nodeCount); - COPY_SCALAR_ARRAY(fileDescriptorArray, int32, from->nodeCount); - - COPY_SCALAR_FIELD(currentNodeIndex); - COPY_SCALAR_FIELD(querySourceNodeIndex); - COPY_SCALAR_FIELD(failureCount); -} - - void CopyNodeDeferredErrorMessage(COPYFUNC_ARGS) { diff --git a/src/backend/distributed/utils/citus_nodefuncs.c b/src/backend/distributed/utils/citus_nodefuncs.c index 20442b3e7..2efd9b003 100644 --- a/src/backend/distributed/utils/citus_nodefuncs.c +++ b/src/backend/distributed/utils/citus_nodefuncs.c @@ -40,7 +40,6 @@ static const char *CitusNodeTagNamesD[] = { "UsedDistributedSubPlan", "Task", "LocalPlannedStatement", - "TaskExecution", "ShardInterval", "ShardPlacement", "RelationShard", @@ -343,7 +342,6 @@ Datum citus_extradata_container(PG_FUNCTION_ARGS) { ereport(ERROR, (errmsg("not supposed to get here, did you cheat?"))); - PG_RETURN_NULL(); } @@ -400,7 +398,6 @@ const ExtensibleNodeMethods nodeMethods[] = DEFINE_NODE_METHODS(RelationRowLock), DEFINE_NODE_METHODS(Task), DEFINE_NODE_METHODS(LocalPlannedStatement), - DEFINE_NODE_METHODS(TaskExecution), DEFINE_NODE_METHODS(DeferredErrorMessage), DEFINE_NODE_METHODS(GroupShardPlacement), diff --git a/src/backend/distributed/utils/citus_outfuncs.c b/src/backend/distributed/utils/citus_outfuncs.c index 903b94e94..265faddf6 100644 --- a/src/backend/distributed/utils/citus_outfuncs.c +++ b/src/backend/distributed/utils/citus_outfuncs.c @@ -521,7 +521,6 @@ OutTask(OUTFUNC_ARGS) WRITE_UINT_FIELD(upstreamTaskId); WRITE_NODE_FIELD(shardInterval); WRITE_BOOL_FIELD(assignmentConstrained); - WRITE_NODE_FIELD(taskExecution); WRITE_CHAR_FIELD(replicationModel); WRITE_BOOL_FIELD(modifyWithSubquery); WRITE_NODE_FIELD(relationShardList); @@ -544,28 +543,6 @@ OutLocalPlannedStatement(OUTFUNC_ARGS) WRITE_NODE_FIELD(localPlan); } - -void -OutTaskExecution(OUTFUNC_ARGS) -{ - WRITE_LOCALS(TaskExecution); - WRITE_NODE_TYPE("TASKEXECUTION"); - - WRITE_UINT64_FIELD(jobId); - WRITE_UINT_FIELD(taskId); - WRITE_UINT_FIELD(nodeCount); - - WRITE_ENUM_ARRAY(taskStatusArray, node->nodeCount); - WRITE_ENUM_ARRAY(transmitStatusArray, node->nodeCount); - WRITE_INT_ARRAY(connectionIdArray, node->nodeCount); - WRITE_INT_ARRAY(fileDescriptorArray, node->nodeCount); - - WRITE_UINT_FIELD(currentNodeIndex); - WRITE_UINT_FIELD(querySourceNodeIndex); - WRITE_UINT_FIELD(failureCount); -} - - void OutDeferredErrorMessage(OUTFUNC_ARGS) { diff --git a/src/backend/distributed/utils/multi_resowner.c b/src/backend/distributed/utils/multi_resowner.c deleted file mode 100644 index 86898ffba..000000000 --- a/src/backend/distributed/utils/multi_resowner.c +++ /dev/null @@ -1,150 +0,0 @@ -/*------------------------------------------------------------------------- - * - * multi_resowner.c - * Citus resource owner integration - * - * An extension can't directly add members to ResourceOwnerData. Instead we - * have to use the resource owner callback mechanism. Right now it's - * sufficient to have an array of referenced resources - there bascially are - * never more than a handful of entries, if that. If that changes we should - * probably rather use a hash table using the pointer value of the resource - * owner as key. - * - * Copyright (c) Citus Data, Inc. - *------------------------------------------------------------------------- - */ - -#include "postgres.h" - -#include "distributed/multi_server_executor.h" -#include "utils/memutils.h" -#include "utils/resowner_private.h" -#include "distributed/multi_resowner.h" -#include "distributed/subplan_execution.h" - - -typedef struct JobDirectoryEntry -{ - ResourceOwner owner; - uint64 jobId; -} JobDirectoryEntry; - - -static bool RegisteredResownerCallback = false; -JobDirectoryEntry *RegisteredJobDirectories = NULL; -size_t NumRegisteredJobDirectories = 0; -size_t NumAllocatedJobDirectories = 0; - - -/* - * Resource owner callback - release resources still held by the resource - * owner. - */ -static void -MultiResourceOwnerReleaseCallback(ResourceReleasePhase phase, - bool isCommit, - bool isTopLevel, - void *arg) -{ - int lastJobIndex = NumRegisteredJobDirectories - 1; - int jobIndex = 0; - - if (phase == RESOURCE_RELEASE_AFTER_LOCKS) - { - /* - * Remove all remaining job directories, after locks have been - * released. - */ - for (jobIndex = lastJobIndex; jobIndex >= 0; jobIndex--) - { - JobDirectoryEntry *entry = &RegisteredJobDirectories[jobIndex]; - - if (entry->owner == CurrentResourceOwner) - { - RemoveJobDirectory(entry->jobId); - } - } - } -} - - -/* - * ResourceOwnerEnlargeJobDirectories makes sure that there is space to - * reference at least one more job directory for the resource owner. Note that - * we only expect one job directory per portal, but we still use an array - * here. - * - * This function is separate from the one actually inserting an entry because - * if we run out of memory, it's critical to do so *before* acquiring the - * resource. - */ -void -ResourceOwnerEnlargeJobDirectories(ResourceOwner owner) -{ - int newMax = 0; - - /* ensure callback is registered */ - if (!RegisteredResownerCallback) - { - RegisterResourceReleaseCallback(MultiResourceOwnerReleaseCallback, NULL); - RegisteredResownerCallback = true; - } - - if (RegisteredJobDirectories == NULL) - { - newMax = 16; - RegisteredJobDirectories = - (JobDirectoryEntry *) MemoryContextAlloc(TopMemoryContext, - newMax * sizeof(JobDirectoryEntry)); - NumAllocatedJobDirectories = newMax; - } - else if (NumRegisteredJobDirectories + 1 > NumAllocatedJobDirectories) - { - newMax = NumAllocatedJobDirectories * 2; - RegisteredJobDirectories = - (JobDirectoryEntry *) repalloc(RegisteredJobDirectories, - newMax * sizeof(JobDirectoryEntry)); - NumAllocatedJobDirectories = newMax; - } -} - - -/* Remembers that a temporary job directory is owned by a resource owner. */ -void -ResourceOwnerRememberJobDirectory(ResourceOwner owner, uint64 jobId) -{ - Assert(NumRegisteredJobDirectories + 1 <= NumAllocatedJobDirectories); - JobDirectoryEntry *entry = &RegisteredJobDirectories[NumRegisteredJobDirectories]; - entry->owner = owner; - entry->jobId = jobId; - NumRegisteredJobDirectories++; -} - - -/* Forgets that a temporary job directory is owned by a resource owner. */ -void -ResourceOwnerForgetJobDirectory(ResourceOwner owner, uint64 jobId) -{ - int lastJobIndex = NumRegisteredJobDirectories - 1; - - for (int jobIndex = lastJobIndex; jobIndex >= 0; jobIndex--) - { - JobDirectoryEntry *entry = &RegisteredJobDirectories[jobIndex]; - - if (entry->owner == owner && entry->jobId == jobId) - { - /* move all later entries one up */ - while (jobIndex < lastJobIndex) - { - RegisteredJobDirectories[jobIndex] = - RegisteredJobDirectories[jobIndex + 1]; - jobIndex++; - } - NumRegisteredJobDirectories = lastJobIndex; - return; - } - } - - elog(ERROR, "jobId " UINT64_FORMAT " is not owned by resource owner %p", - jobId, owner); -} diff --git a/src/backend/distributed/utils/task_execution_utils.c b/src/backend/distributed/utils/task_execution_utils.c new file mode 100644 index 000000000..92b0c3ea2 --- /dev/null +++ b/src/backend/distributed/utils/task_execution_utils.c @@ -0,0 +1,224 @@ + +#include "postgres.h" +#include "miscadmin.h" + +#include +#include +#include + +#include "commands/dbcommands.h" +#include "distributed/citus_custom_scan.h" +#include "distributed/citus_nodes.h" +#include "distributed/connection_management.h" +#include "distributed/deparse_shard_query.h" +#include "distributed/distributed_execution_locks.h" +#include "distributed/listutils.h" +#include "distributed/local_executor.h" +#include "distributed/metadata_cache.h" +#include "distributed/multi_client_executor.h" +#include "distributed/multi_executor.h" +#include "distributed/multi_physical_planner.h" +#include "distributed/multi_server_executor.h" +#include "distributed/pg_dist_partition.h" +#include "distributed/resource_lock.h" +#include "distributed/subplan_execution.h" +#include "distributed/task_execution_utils.h" +#include "distributed/worker_protocol.h" +#include "distributed/version_compat.h" +#include "storage/fd.h" +#include "utils/builtins.h" +#include "utils/hsearch.h" +#include "utils/timestamp.h" + +/* TaskMapKey is used as a key in task hash */ +typedef struct TaskMapKey +{ + TaskType taskType; + uint64 jobId; + uint32 taskId; +} TaskMapKey; + + +/* + * TaskMapEntry is used as entry in task hash. We need to keep a pointer + * of the task in the entry. + */ +typedef struct TaskMapEntry +{ + TaskMapKey key; + Task *task; +} TaskMapEntry; + +static HTAB * TaskHashCreate(uint32 taskHashSize); +static Task * TaskHashEnter(HTAB *taskHash, Task *task); +static Task * TaskHashLookup(HTAB *trackerHash, TaskType taskType, uint64 jobId, + uint32 taskId); + +/* + * CreateTaskListForJobTree visits all tasks in the job tree (by following dependentTaskList), + * starting with the given job's task list. The function then returns the list. + */ +List * +CreateTaskListForJobTree(List *jobTaskList) +{ + List *taskList = NIL; + const int topLevelTaskHashSize = 32; + int taskHashSize = list_length(jobTaskList) * topLevelTaskHashSize; + HTAB *taskHash = TaskHashCreate(taskHashSize); + + /* + * We walk over the task tree using breadth-first search. For the search, we + * first queue top level tasks in the task tree. + */ + List *taskQueue = list_copy(jobTaskList); + while (taskQueue != NIL) + { + /* pop first element from the task queue */ + Task *task = (Task *) linitial(taskQueue); + taskQueue = list_delete_first(taskQueue); + + taskList = lappend(taskList, task); + + List *dependendTaskList = task->dependentTaskList; + + /* + * Push task node's children into the task queue, if and only if + * they're not already there. As task dependencies have to form a + * directed-acyclic-graph and are processed in a breadth-first search + * we can never re-encounter nodes we've already processed. + * + * While we're checking this, we can also fix the problem that + * copyObject() might have duplicated nodes in the graph - if a node + * isn't pushed to the graph because it is already planned to be + * visited, we can simply replace it with the copy. Note that, here + * we only consider dependend tasks. Since currently top level tasks + * cannot be on any dependend task list, we do not check them for duplicates. + * + * taskHash is used to reduce the complexity of keeping track of + * the tasks that are already encountered. + */ + ListCell *dependentTaskCell = NULL; + foreach(dependentTaskCell, dependendTaskList) + { + Task *dependendTask = lfirst(dependentTaskCell); + Task *dependendTaskInHash = TaskHashLookup(taskHash, + dependendTask->taskType, + dependendTask->jobId, + dependendTask->taskId); + + /* + * If the dependend task encountered for the first time, add it to the hash. + * Also, add this task to the task queue. Note that, we do not need to + * add the tasks to the queue which are already encountered, because + * they are already added to the queue. + */ + if (!dependendTaskInHash) + { + dependendTaskInHash = TaskHashEnter(taskHash, dependendTask); + taskQueue = lappend(taskQueue, dependendTaskInHash); + } + + /* update dependentTaskList element to the one which is in the hash */ + lfirst(dependentTaskCell) = dependendTaskInHash; + } + } + + return taskList; +} + + +/* + * TaskHashCreate allocates memory for a task hash, initializes an + * empty hash, and returns this hash. + */ +static HTAB * +TaskHashCreate(uint32 taskHashSize) +{ + HASHCTL info; + const char *taskHashName = "Task Hash"; + + /* + * Can't create a hashtable of size 0. Normally that shouldn't happen, but + * shard pruning currently can lead to this (Job with 0 Tasks). See #833. + */ + if (taskHashSize == 0) + { + taskHashSize = 2; + } + + memset(&info, 0, sizeof(info)); + info.keysize = sizeof(TaskMapKey); + info.entrysize = sizeof(TaskMapEntry); + info.hash = tag_hash; + info.hcxt = CurrentMemoryContext; + int hashFlags = (HASH_ELEM | HASH_FUNCTION | HASH_CONTEXT); + + HTAB *taskHash = hash_create(taskHashName, taskHashSize, &info, hashFlags); + + return taskHash; +} + + +/* + * TaskHashEnter creates a reference to the task entry in the given task + * hash. The function errors-out if the same key exists multiple times. + */ +static Task * +TaskHashEnter(HTAB *taskHash, Task *task) +{ + bool handleFound = false; + + TaskMapKey taskKey; + memset(&taskKey, 0, sizeof(TaskMapKey)); + + taskKey.taskType = task->taskType; + taskKey.jobId = task->jobId; + taskKey.taskId = task->taskId; + + void *hashKey = (void *) &taskKey; + TaskMapEntry *taskInTheHash = (TaskMapEntry *) hash_search(taskHash, hashKey, + HASH_ENTER, + &handleFound); + + /* if same node appears twice, we error-out */ + if (handleFound) + { + ereport(ERROR, (errmsg("multiple entries for task: \"%d:" UINT64_FORMAT ":%u\"", + task->taskType, task->jobId, task->taskId))); + } + + /* save the pointer to the original task in the hash */ + taskInTheHash->task = task; + + return task; +} + + +/* + * TaskHashLookup looks for the tasks that corresponds to the given + * taskType, jobId and taskId, and returns the found task, NULL otherwise. + */ +static Task * +TaskHashLookup(HTAB *taskHash, TaskType taskType, uint64 jobId, uint32 taskId) +{ + Task *task = NULL; + bool handleFound = false; + + TaskMapKey taskKey; + memset(&taskKey, 0, sizeof(TaskMapKey)); + + taskKey.taskType = taskType; + taskKey.jobId = jobId; + taskKey.taskId = taskId; + + void *hashKey = (void *) &taskKey; + TaskMapEntry *taskEntry = (TaskMapEntry *) hash_search(taskHash, hashKey, HASH_FIND, + &handleFound); + + if (taskEntry != NULL) + { + task = taskEntry->task; + } + + return task; +} diff --git a/src/backend/distributed/worker/task_tracker.c b/src/backend/distributed/worker/task_tracker.c deleted file mode 100644 index 0a72579f1..000000000 --- a/src/backend/distributed/worker/task_tracker.c +++ /dev/null @@ -1,1108 +0,0 @@ -/*------------------------------------------------------------------------- - * - * task_tracker.c - * - * The task tracker background process runs on every worker node. The process - * wakes up at regular intervals, reads information from a shared hash, and - * checks if any new tasks are assigned to this node. If they are, the process - * runs task-specific logic, and sends queries to the postmaster for execution. - * The task tracker then tracks the execution of these queries, and updates the - * shared hash with task progress information. - * - * The task tracker is started by the postmaster when the startup process - * finishes. The process remains alive until the postmaster commands it to - * terminate. Normal termination is by SIGTERM, which instructs the task tracker - * to exit(0). Emergency termination is by SIGQUIT; like any backend, the task - * tracker will simply abort and exit on SIGQUIT. - * - * For details on how the task tracker manages resources during process start-up - * and shutdown, please see the writeboard on our Basecamp project website. - * - * Copyright (c) Citus Data, Inc. - * - * $Id$ - * - *------------------------------------------------------------------------- - */ - -#include "postgres.h" -#include "miscadmin.h" -#include - -#include "commands/dbcommands.h" -#include "distributed/citus_safe_lib.h" -#include "distributed/listutils.h" -#include "distributed/multi_client_executor.h" -#include "distributed/multi_server_executor.h" -#include "distributed/task_tracker.h" -#include "distributed/transmit.h" -#include "distributed/worker_protocol.h" -#include "distributed/version_compat.h" -#include "libpq/hba.h" -#include "libpq/pqsignal.h" -#include "lib/stringinfo.h" -#include "postmaster/bgworker.h" -#include "postmaster/postmaster.h" -#include "storage/fd.h" -#include "storage/ipc.h" -#include "storage/lwlock.h" -#include "storage/pmsignal.h" -#include "storage/proc.h" -#include "storage/shmem.h" -#include "utils/guc.h" -#include "utils/memutils.h" - - -int TaskTrackerDelay = 200; /* process sleep interval in millisecs */ -int MaxRunningTasksPerNode = 16; /* max number of running tasks */ -int MaxTrackedTasksPerNode = 1024; /* max number of tracked tasks */ -int MaxTaskStringSize = 12288; /* max size of a worker task call string in bytes */ -WorkerTasksSharedStateData *WorkerTasksSharedState; /* shared memory state */ - -/* Hash table shared by the task tracker and task tracker protocol functions */ -HTAB *TaskTrackerTaskHash = NULL; /* shared memory */ - -static shmem_startup_hook_type prev_shmem_startup_hook = NULL; - -/* Flags set by interrupt handlers for later service in the main loop */ -static volatile sig_atomic_t got_SIGHUP = false; -static volatile sig_atomic_t got_SIGTERM = false; - -/* initialization forward declarations */ -static Size TaskTrackerShmemSize(void); -static void TaskTrackerShmemInit(void); - -/* Signal handler forward declarations */ -static void TrackerSigHupHandler(SIGNAL_ARGS); -static void TrackerShutdownHandler(SIGNAL_ARGS); - -/* Local functions forward declarations */ -static void TrackerCleanupJobSchemas(void); -static void TrackerCleanupConnections(HTAB *WorkerTasksHash); -static void TrackerRegisterShutDown(HTAB *WorkerTasksHash); -static void TrackerDelayLoop(void); -static List * SchedulableTaskList(HTAB *WorkerTasksHash); -static WorkerTask * SchedulableTaskPriorityQueue(HTAB *WorkerTasksHash); -static uint32 CountTasksMatchingCriteria(HTAB *WorkerTasksHash, - bool (*CriteriaFunction)(WorkerTask *)); -static bool RunningTask(WorkerTask *workerTask); -static bool SchedulableTask(WorkerTask *workerTask); -static int CompareTasksByTime(const void *first, const void *second); -static void ScheduleWorkerTasks(HTAB *WorkerTasksHash, List *schedulableTaskList); -static void ManageWorkerTasksHash(HTAB *WorkerTasksHash); -static void ManageWorkerTask(WorkerTask *workerTask, HTAB *WorkerTasksHash); -static void RemoveWorkerTask(WorkerTask *workerTask, HTAB *WorkerTasksHash); -static void CreateJobDirectoryIfNotExists(uint64 jobId); -static void MarkWorkerTaskAsFailed(WorkerTask *workerTask); -static int32 ConnectToLocalBackend(const char *databaseName, const char *userName); - - -/* Organize, at startup, that the task tracker is started */ -void -TaskTrackerRegister(void) -{ - BackgroundWorker worker; - - prev_shmem_startup_hook = shmem_startup_hook; - shmem_startup_hook = TaskTrackerShmemInit; - - if (IsUnderPostmaster) - { - return; - } - - /* organize and register initialization of required shared memory */ - RequestAddinShmemSpace(TaskTrackerShmemSize()); - - /* and that the task tracker is started as background worker */ - memset(&worker, 0, sizeof(worker)); - worker.bgw_flags = BGWORKER_SHMEM_ACCESS; - worker.bgw_start_time = BgWorkerStart_ConsistentState; - worker.bgw_restart_time = 1; - strcpy_s(worker.bgw_library_name, sizeof(worker.bgw_library_name), "citus"); - strcpy_s(worker.bgw_function_name, sizeof(worker.bgw_function_name), - "TaskTrackerMain"); - worker.bgw_notify_pid = 0; - strcpy_s(worker.bgw_name, sizeof(worker.bgw_name), "task tracker"); - - RegisterBackgroundWorker(&worker); -} - - -/* Main entry point for task tracker process. */ -void -TaskTrackerMain(Datum main_arg) -{ - sigjmp_buf local_sigjmp_buf; - static bool processStartUp = true; - - /* Properly accept or ignore signals the postmaster might send us */ - pqsignal(SIGHUP, TrackerSigHupHandler); /* set flag to read config file */ - pqsignal(SIGTERM, TrackerShutdownHandler); /* request shutdown */ - - /* We're now ready to receive signals */ - BackgroundWorkerUnblockSignals(); - - /* - * Create a memory context that we will do all our work in. We do this so - * that we can reset the context during error recovery and thereby avoid - * possible memory leaks. - */ - MemoryContext TaskTrackerContext = AllocSetContextCreateExtended(TopMemoryContext, - "Task Tracker", - ALLOCSET_DEFAULT_MINSIZE, - ALLOCSET_DEFAULT_INITSIZE, - ALLOCSET_DEFAULT_MAXSIZE); - MemoryContextSwitchTo(TaskTrackerContext); - - /* - * If an exception is encountered, processing resumes here. The motivation - * for this code block is outlined in postgres.c, and the code itself is - * heavily based on bgwriter.c. - * - * In most error scenarios, we will not drop here: the task tracker process - * offloads all work to backend processes, and checks the completion of work - * through the client executor library. We will therefore only come here if - * we have inconsistencies in the shared hash and need to signal an error. - */ - if (sigsetjmp(local_sigjmp_buf, 1) != 0) - { - /* Since we are not using PG_TRY, we must reset error stack by hand */ - error_context_stack = NULL; - - /* Prevents interrupts while cleaning up */ - HOLD_INTERRUPTS(); - - /* Report the error to the server log */ - EmitErrorReport(); - - /* - * These operations are just a minimal subset of AbortTransaction(). - * We do not have many resources to worry about; we only have a shared - * hash and an LWLock guarding that hash. - */ - LWLockReleaseAll(); - AtEOXact_Files(false); - AtEOXact_HashTables(false); - - /* - * Now return to normal top-level context, and clear ErrorContext for - * next time. - */ - MemoryContextSwitchTo(TaskTrackerContext); - FlushErrorState(); - - /* Flush any leaked data in the top-level context */ - MemoryContextResetAndDeleteChildren(TaskTrackerContext); - - /* Now we can allow interrupts again */ - RESUME_INTERRUPTS(); - - /* - * Sleep at least 1 second after any error. A write error is likely to - * be repeated, and we don't want to be filling the error logs as fast - * as we can. - */ - pg_usleep(1000000L); - } - - /* We can now handle ereport(ERROR) */ - PG_exception_stack = &local_sigjmp_buf; - - /* - * We run validation and cache cleanup functions as this process is starting - * up. If these functions throw an error, we won't try running them again. - */ - if (processStartUp) - { - processStartUp = false; - - /* clean up old files in the job cache */ - TrackerCleanupJobDirectories(); - - /* clean up schemas in the job cache */ - TrackerCleanupJobSchemas(); - } - - /* Loop forever */ - for (;;) - { - /* - * Emergency bailout if postmaster has died. This is to avoid the - * necessity for manual cleanup of all postmaster children. - * - * XXX: Note that PostgreSQL background processes no longer nap between - * their loops, but instead uses latches to wake up when necessary. We - * should switch to using latches in here too, and have the task tracker - * assign function notify us when there is a new task. - */ - if (!PostmasterIsAlive()) - { - exit(1); - } - - /* Process any requests or signals received recently */ - if (got_SIGHUP) - { - got_SIGHUP = false; - - /* reload postgres configuration files */ - ProcessConfigFile(PGC_SIGHUP); - } - if (got_SIGTERM) - { - /* - * From here on, reporting errors should end with exit(1), and not - * send control back to the sigsetjmp block above. - */ - ExitOnAnyError = true; - - /* Close open connections to local backends */ - TrackerCleanupConnections(TaskTrackerTaskHash); - - /* Add a sentinel task to the shared hash to mark shutdown */ - TrackerRegisterShutDown(TaskTrackerTaskHash); - - /* Normal exit from the task tracker is here */ - proc_exit(0); - } - - /* Call the function that does the actual work */ - ManageWorkerTasksHash(TaskTrackerTaskHash); - - /* Sleep for the configured time */ - TrackerDelayLoop(); - } -} - - -/* - * WorkerTasksHashEnter creates a new worker task in the shared hash, and - * performs checks for this task. Note that the caller still needs to initialize - * the worker task's fields, and hold the appopriate locks for the shared hash. - */ -WorkerTask * -WorkerTasksHashEnter(uint64 jobId, uint32 taskId) -{ - bool handleFound = false; - - WorkerTask searchTask; - searchTask.jobId = jobId; - searchTask.taskId = taskId; - - void *hashKey = (void *) &searchTask; - WorkerTask *workerTask = (WorkerTask *) hash_search(TaskTrackerTaskHash, hashKey, - HASH_ENTER_NULL, &handleFound); - if (workerTask == NULL) - { - ereport(ERROR, (errcode(ERRCODE_OUT_OF_MEMORY), - errmsg("out of shared memory"), - errhint("Try increasing citus.max_tracked_tasks_per_node."))); - } - - /* check that we do not have the same task assigned twice to this node */ - if (handleFound) - { - ereport(ERROR, (errmsg("cannot assign an already assigned task"), - errdetail("Task jobId: " UINT64_FORMAT " and taskId: %u", - jobId, taskId))); - } - - return workerTask; -} - - -/* - * WorkerTasksHashFind looks up the worker task with the given identifiers in - * the shared hash. Note that the caller still needs to hold the appropriate - * locks for the shared hash. - */ -WorkerTask * -WorkerTasksHashFind(uint64 jobId, uint32 taskId) -{ - WorkerTask searchTask; - searchTask.jobId = jobId; - searchTask.taskId = taskId; - - void *hashKey = (void *) &searchTask; - WorkerTask *workerTask = (WorkerTask *) hash_search(TaskTrackerTaskHash, hashKey, - HASH_FIND, NULL); - - return workerTask; -} - - -/* - * TrackerCleanupJobDirectories cleans up all files in the job cache directory - * as part of this process's start-up logic. The task tracker process manages - * both tasks in the shared hash and these tasks' output files. When the task - * tracker needs to shutdown, all shared hash entries are deleted, but the - * associated files cannot be cleaned up safely. We therefore perform this - * cleanup when the process restarts. - */ -void -TrackerCleanupJobDirectories(void) -{ - /* use the default tablespace in {datadir}/base */ - StringInfo jobCacheDirectory = makeStringInfo(); - appendStringInfo(jobCacheDirectory, "base/%s", PG_JOB_CACHE_DIR); - - CitusRemoveDirectory(jobCacheDirectory->data); - CitusCreateDirectory(jobCacheDirectory); - - FreeStringInfo(jobCacheDirectory); -} - - -/* - * TrackerCleanupJobSchemas creates and assigns tasks to remove job schemas and - * all tables within these schemas. These job schemas are currently created by - * merge tasks, and may linger if the database shuts down before the jobs get - * cleaned up. This function then runs during process start-up, and creates one - * task per database to remove lingering job schemas, if any. - */ -static void -TrackerCleanupJobSchemas(void) -{ - /* - * XXX: We previously called DatabaseNameList() to read the list of database - * names here. This function read the database names from the flat database - * file; this file was deprecated on Aug 31, 2009. We hence need to rewrite - * this function to read from pg_database directly. - */ - List *databaseNameList = NIL; - const uint64 jobId = RESERVED_JOB_ID; - uint32 taskIndex = 1; - - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_EXCLUSIVE); - - const char *databaseName = NULL; - foreach_ptr(databaseName, databaseNameList) - { - /* template0 database does not accept connections */ - int skipDatabaseName = strncmp(databaseName, TEMPLATE0_NAME, NAMEDATALEN); - if (skipDatabaseName == 0) - { - continue; - } - - /* - * We create cleanup tasks since we can't remove schemas within the task - * tracker process. We also assign high priorities to these tasks so - * that they get scheduled before everyone else. - */ - WorkerTask *cleanupTask = WorkerTasksHashEnter(jobId, taskIndex); - cleanupTask->assignedAt = HIGH_PRIORITY_TASK_TIME; - cleanupTask->taskStatus = TASK_ASSIGNED; - - strlcpy(cleanupTask->taskCallString, JOB_SCHEMA_CLEANUP, MaxTaskStringSize); - strlcpy(cleanupTask->databaseName, databaseName, NAMEDATALEN); - - /* zero out all other fields */ - cleanupTask->connectionId = INVALID_CONNECTION_ID; - cleanupTask->failureCount = 0; - - taskIndex++; - } - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); - - if (databaseNameList != NIL) - { - list_free_deep(databaseNameList); - } -} - - -/* - * TrackerCleanupConnections closes all open connections to backends during - * process shutdown. This signals to the backends that their connections are - * gone and stops them from logging pipe-related warning messages. - */ -static void -TrackerCleanupConnections(HTAB *WorkerTasksHash) -{ - HASH_SEQ_STATUS status; - - hash_seq_init(&status, WorkerTasksHash); - - WorkerTask *currentTask = (WorkerTask *) hash_seq_search(&status); - while (currentTask != NULL) - { - if (currentTask->connectionId != INVALID_CONNECTION_ID) - { - MultiClientDisconnect(currentTask->connectionId); - currentTask->connectionId = INVALID_CONNECTION_ID; - } - - currentTask = (WorkerTask *) hash_seq_search(&status); - } -} - - -/* - * TrackerRegisterShutDown enters a special marker task to the shared hash. This - * marker task indicates to "task protocol processes" that we are shutting down - * and that they shouldn't accept new task assignments. - */ -static void -TrackerRegisterShutDown(HTAB *WorkerTasksHash) -{ - uint64 jobId = RESERVED_JOB_ID; - uint32 taskId = SHUTDOWN_MARKER_TASK_ID; - - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_EXCLUSIVE); - - WorkerTask *shutdownMarkerTask = WorkerTasksHashEnter(jobId, taskId); - shutdownMarkerTask->taskStatus = TASK_SUCCEEDED; - shutdownMarkerTask->connectionId = INVALID_CONNECTION_ID; - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); -} - - -/* Sleeps either for the configured time or until a signal is received. */ -static void -TrackerDelayLoop(void) -{ - const long SignalCheckInterval = 1000000L; /* check signal every second */ - - /* - * On some platforms, signals do not interrupt the sleep. To ensure we - * respond promptly when someone signals us, we break down the sleep into - * 1-second increments, and check for interrupts after each nap. - */ - long trackerDelay = TaskTrackerDelay * 1000L; - while (trackerDelay > (SignalCheckInterval - 1)) - { - if (got_SIGHUP || got_SIGTERM) - { - break; - } - pg_usleep(SignalCheckInterval); - trackerDelay -= SignalCheckInterval; - } - if (!(got_SIGHUP || got_SIGTERM)) - { - pg_usleep(trackerDelay); - } -} - - -/* ------------------------------------------------------------ - * Signal handling and shared hash initialization functions follow - * ------------------------------------------------------------ - */ - -/* SIGHUP: set flag to re-read config file at next convenient time */ -static void -TrackerSigHupHandler(SIGNAL_ARGS) -{ - int save_errno = errno; - - got_SIGHUP = true; - if (MyProc != NULL) - { - SetLatch(&MyProc->procLatch); - } - - errno = save_errno; -} - - -/* SIGTERM: set flag for main loop to exit normally */ -static void -TrackerShutdownHandler(SIGNAL_ARGS) -{ - int save_errno = errno; - - got_SIGTERM = true; - if (MyProc != NULL) - { - SetLatch(&MyProc->procLatch); - } - - errno = save_errno; -} - - -/* Estimates the shared memory size used for keeping track of tasks. */ -static Size -TaskTrackerShmemSize(void) -{ - Size size = 0; - - size = add_size(size, sizeof(WorkerTasksSharedStateData)); - - Size hashSize = hash_estimate_size(MaxTrackedTasksPerNode, WORKER_TASK_SIZE); - size = add_size(size, hashSize); - - return size; -} - - -/* Initializes the shared memory used for keeping track of tasks. */ -static void -TaskTrackerShmemInit(void) -{ - bool alreadyInitialized = false; - HASHCTL info; - - long maxTableSize = (long) MaxTrackedTasksPerNode; - long initTableSize = maxTableSize / 8; - - /* - * Allocate the control structure for the hash table that maps unique task - * identifiers (uint64:uint32) to general task information, as well as the - * parameters needed to run the task. - */ - memset(&info, 0, sizeof(info)); - info.keysize = sizeof(uint64) + sizeof(uint32); - info.entrysize = WORKER_TASK_SIZE; - info.hash = tag_hash; - int hashFlags = (HASH_ELEM | HASH_FUNCTION); - - /* - * Currently the lock isn't required because allocation only happens at - * startup in postmaster, but it doesn't hurt, and makes things more - * consistent with other extensions. - */ - LWLockAcquire(AddinShmemInitLock, LW_EXCLUSIVE); - - /* allocate struct containing task tracker related shared state */ - WorkerTasksSharedState = - (WorkerTasksSharedStateData *) ShmemInitStruct("Worker Task Control", - sizeof(WorkerTasksSharedStateData), - &alreadyInitialized); - - if (!alreadyInitialized) - { - WorkerTasksSharedState->taskHashTrancheId = LWLockNewTrancheId(); - WorkerTasksSharedState->taskHashTrancheName = "Worker Task Hash Tranche"; - LWLockRegisterTranche(WorkerTasksSharedState->taskHashTrancheId, - WorkerTasksSharedState->taskHashTrancheName); - - LWLockInitialize(&WorkerTasksSharedState->taskHashLock, - WorkerTasksSharedState->taskHashTrancheId); - - WorkerTasksSharedState->conninfosValid = true; - } - - /* allocate hash table */ - TaskTrackerTaskHash = ShmemInitHash("Worker Task Hash", initTableSize, maxTableSize, - &info, hashFlags); - - LWLockRelease(AddinShmemInitLock); - - Assert(TaskTrackerTaskHash != NULL); - Assert(WorkerTasksSharedState->taskHashTrancheId != 0); - - if (prev_shmem_startup_hook != NULL) - { - prev_shmem_startup_hook(); - } -} - - -/* ------------------------------------------------------------ - * Task scheduling and management functions follow - * ------------------------------------------------------------ - */ - -/* - * SchedulableTaskList calculates the number of tasks to schedule at this given - * moment, and creates a deep-copied list containing that many tasks. The tasks - * in the list are sorted according to a priority criteria, currently the task's - * assignment time. Note that this function expects the caller to hold a read - * lock over the shared hash. - */ -static List * -SchedulableTaskList(HTAB *WorkerTasksHash) -{ - List *schedulableTaskList = NIL; - - uint32 runningTaskCount = CountTasksMatchingCriteria(WorkerTasksHash, &RunningTask); - if (runningTaskCount >= MaxRunningTasksPerNode) - { - return NIL; /* we already have enough tasks running */ - } - - uint32 schedulableTaskCount = CountTasksMatchingCriteria(WorkerTasksHash, - &SchedulableTask); - if (schedulableTaskCount == 0) - { - return NIL; /* we do not have any new tasks to schedule */ - } - - uint32 tasksToScheduleCount = MaxRunningTasksPerNode - runningTaskCount; - if (tasksToScheduleCount > schedulableTaskCount) - { - tasksToScheduleCount = schedulableTaskCount; - } - - /* get all schedulable tasks ordered according to a priority criteria */ - WorkerTask *schedulableTaskQueue = SchedulableTaskPriorityQueue(WorkerTasksHash); - if (schedulableTaskQueue == NULL) - { - return NIL; - } - - for (uint32 queueIndex = 0; queueIndex < tasksToScheduleCount; queueIndex++) - { - WorkerTask *schedulableTask = (WorkerTask *) palloc0(WORKER_TASK_SIZE); - WorkerTask *queuedTask = WORKER_TASK_AT(schedulableTaskQueue, queueIndex); - schedulableTask->jobId = queuedTask->jobId; - schedulableTask->taskId = queuedTask->taskId; - - schedulableTaskList = lappend(schedulableTaskList, schedulableTask); - } - - /* free priority queue */ - pfree(schedulableTaskQueue); - - return schedulableTaskList; -} - - -/* - * SchedulableTaskPriorityQueue allocates an array containing all schedulable - * tasks in the shared hash, orders these tasks according to a sorting criteria, - * and returns the sorted array. - */ -static WorkerTask * -SchedulableTaskPriorityQueue(HTAB *WorkerTasksHash) -{ - HASH_SEQ_STATUS status; - uint32 queueIndex = 0; - - /* our priority queue size equals to the number of schedulable tasks */ - uint32 queueSize = CountTasksMatchingCriteria(WorkerTasksHash, &SchedulableTask); - if (queueSize == 0) - { - return NULL; - } - - /* allocate an array of tasks for our priority queue */ - WorkerTask *priorityQueue = (WorkerTask *) palloc0(WORKER_TASK_SIZE * queueSize); - - /* copy tasks in the shared hash to the priority queue */ - hash_seq_init(&status, WorkerTasksHash); - - WorkerTask *currentTask = (WorkerTask *) hash_seq_search(&status); - while (currentTask != NULL) - { - if (SchedulableTask(currentTask)) - { - /* tasks in the priority queue only need the first three fields */ - WorkerTask *queueTask = WORKER_TASK_AT(priorityQueue, queueIndex); - - queueTask->jobId = currentTask->jobId; - queueTask->taskId = currentTask->taskId; - queueTask->assignedAt = currentTask->assignedAt; - - queueIndex++; - } - - currentTask = (WorkerTask *) hash_seq_search(&status); - } - - /* now order elements in the queue according to our sorting criterion */ - SafeQsort(priorityQueue, queueSize, WORKER_TASK_SIZE, CompareTasksByTime); - - return priorityQueue; -} - - -/* Counts the number of tasks that match the given criteria function. */ -static uint32 -CountTasksMatchingCriteria(HTAB *WorkerTasksHash, - bool (*CriteriaFunction)(WorkerTask *)) -{ - HASH_SEQ_STATUS status; - uint32 taskCount = 0; - - hash_seq_init(&status, WorkerTasksHash); - - WorkerTask *currentTask = (WorkerTask *) hash_seq_search(&status); - while (currentTask != NULL) - { - bool matchesCriteria = (*CriteriaFunction)(currentTask); - if (matchesCriteria) - { - taskCount++; - } - - currentTask = (WorkerTask *) hash_seq_search(&status); - } - - return taskCount; -} - - -/* Checks if the worker task is running. */ -static bool -RunningTask(WorkerTask *workerTask) -{ - TaskStatus currentStatus = workerTask->taskStatus; - if (currentStatus == TASK_RUNNING) - { - return true; - } - - return false; -} - - -/* Checks if the worker task can be scheduled to run. */ -static bool -SchedulableTask(WorkerTask *workerTask) -{ - TaskStatus currentStatus = workerTask->taskStatus; - if (currentStatus == TASK_ASSIGNED) - { - return true; - } - - return false; -} - - -/* Comparison function to compare two worker tasks by their assignment times. */ -static int -CompareTasksByTime(const void *first, const void *second) -{ - WorkerTask *firstTask = (WorkerTask *) first; - WorkerTask *secondTask = (WorkerTask *) second; - - /* tasks that are assigned earlier have higher priority */ - int timeDiff = firstTask->assignedAt - secondTask->assignedAt; - return timeDiff; -} - - -/* - * ScheduleWorkerTasks takes a list of tasks to schedule, and for each task in - * the list, finds and schedules the corresponding task from the shared hash. - * Note that this function expects the caller to hold an exclusive lock over the - * shared hash. - */ -static void -ScheduleWorkerTasks(HTAB *WorkerTasksHash, List *schedulableTaskList) -{ - WorkerTask *schedulableTask = NULL; - foreach_ptr(schedulableTask, schedulableTaskList) - { - void *hashKey = (void *) schedulableTask; - - WorkerTask *taskToSchedule = (WorkerTask *) hash_search(WorkerTasksHash, hashKey, - HASH_FIND, NULL); - - /* if task is null, the shared hash is in an incosistent state */ - if (taskToSchedule == NULL) - { - ereport(ERROR, (errmsg("could not find the worker task to schedule"), - errdetail("Task jobId: " UINT64_FORMAT " and taskId: %u", - schedulableTask->jobId, schedulableTask->taskId))); - } - - /* - * After determining the set of tasks to schedule, we release the hash's - * shared lock for a short time period. We then re-acquire the lock in - * exclusive mode. We therefore need to check if this task has been - * canceled in the meantime. - */ - if (taskToSchedule->taskStatus != TASK_CANCEL_REQUESTED) - { - Assert(SchedulableTask(taskToSchedule)); - - taskToSchedule->taskStatus = TASK_SCHEDULED; - } - else - { - ereport(INFO, (errmsg("the worker task to schedule has been canceled"), - errdetail("Task jobId: " UINT64_FORMAT " and taskId: %u", - schedulableTask->jobId, schedulableTask->taskId))); - } - } -} - - -/* Manages the scheduling and execution of all tasks in the shared hash. */ -static void -ManageWorkerTasksHash(HTAB *WorkerTasksHash) -{ - HASH_SEQ_STATUS status; - - /* ask the scheduler if we have new tasks to schedule */ - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_SHARED); - List *schedulableTaskList = SchedulableTaskList(WorkerTasksHash); - LWLockRelease(&WorkerTasksSharedState->taskHashLock); - - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_EXCLUSIVE); - - if (!WorkerTasksSharedState->conninfosValid) - { - InvalidateConnParamsHashEntries(); - } - - /* schedule new tasks if we have any */ - if (schedulableTaskList != NIL) - { - ScheduleWorkerTasks(WorkerTasksHash, schedulableTaskList); - list_free_deep(schedulableTaskList); - } - - /* now iterate over all tasks, and manage them */ - hash_seq_init(&status, WorkerTasksHash); - - WorkerTask *currentTask = (WorkerTask *) hash_seq_search(&status); - while (currentTask != NULL) - { - ManageWorkerTask(currentTask, WorkerTasksHash); - - /* - * Typically, we delete worker tasks in the task tracker protocol - * process. This task however was canceled mid-query, and the protocol - * process asked us to remove it from the shared hash. - */ - if (currentTask->taskStatus == TASK_TO_REMOVE) - { - RemoveWorkerTask(currentTask, WorkerTasksHash); - } - - currentTask = (WorkerTask *) hash_seq_search(&status); - } - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); -} - - -/* - * ManageWorkerTask manages the execution of the worker task. More specifically, - * the function connects to a local backend, sends the query associated with the - * task, and oversees the query's execution. Note that this function expects the - * caller to hold an exclusive lock over the shared hash. - */ -static void -ManageWorkerTask(WorkerTask *workerTask, HTAB *WorkerTasksHash) -{ - switch (workerTask->taskStatus) - { - case TASK_ASSIGNED: - { - break; /* nothing to do until the task gets scheduled */ - } - - case TASK_SCHEDULED: - { - /* create the job output directory if it does not exist */ - CreateJobDirectoryIfNotExists(workerTask->jobId); - - /* the task is ready to run; connect to local backend */ - workerTask->connectionId = ConnectToLocalBackend(workerTask->databaseName, - workerTask->userName); - - if (workerTask->connectionId != INVALID_CONNECTION_ID) - { - bool taskSent = MultiClientSendQuery(workerTask->connectionId, - workerTask->taskCallString); - if (taskSent) - { - workerTask->taskStatus = TASK_RUNNING; - } - else - { - MarkWorkerTaskAsFailed(workerTask); - - MultiClientDisconnect(workerTask->connectionId); - workerTask->connectionId = INVALID_CONNECTION_ID; - } - } - else - { - MarkWorkerTaskAsFailed(workerTask); - } - - break; - } - - case TASK_RUNNING: - { - int32 connectionId = workerTask->connectionId; - ResultStatus resultStatus = MultiClientResultStatus(connectionId); - - /* check if query results are ready, in progress, or unavailable */ - if (resultStatus == CLIENT_RESULT_READY) - { - QueryStatus queryStatus = MultiClientQueryStatus(connectionId); - if (queryStatus == CLIENT_QUERY_DONE) - { - workerTask->taskStatus = TASK_SUCCEEDED; - } - else if (queryStatus == CLIENT_QUERY_FAILED) - { - MarkWorkerTaskAsFailed(workerTask); - } - else - { - ereport(FATAL, (errmsg("invalid query status: %d", queryStatus))); - } - } - else if (resultStatus == CLIENT_RESULT_BUSY) - { - workerTask->taskStatus = TASK_RUNNING; - } - else if (resultStatus == CLIENT_RESULT_UNAVAILABLE) - { - MarkWorkerTaskAsFailed(workerTask); - } - - /* clean up the connection if we are done with the task */ - if (resultStatus != CLIENT_RESULT_BUSY) - { - MultiClientDisconnect(workerTask->connectionId); - workerTask->connectionId = INVALID_CONNECTION_ID; - } - - break; - } - - case TASK_FAILED: - { - if (workerTask->failureCount < MAX_TASK_FAILURE_COUNT) - { - workerTask->taskStatus = TASK_ASSIGNED; - } - else - { - workerTask->taskStatus = TASK_PERMANENTLY_FAILED; - } - - break; - } - - case TASK_PERMANENTLY_FAILED: - case TASK_SUCCEEDED: - { - break; - } - - case TASK_CANCEL_REQUESTED: - { - /* - * If this task is responsible for cleaning up the shared hash, we - * give the task more time instead of canceling it. The reason this - * task is marked for cancellation is that its file descriptor needs - * to be reclaimed after the clean up completes. - */ - if (workerTask->taskId == JOB_CLEANUP_TASK_ID) - { - workerTask->taskStatus = TASK_CANCELED; - break; - } - - if (workerTask->connectionId != INVALID_CONNECTION_ID) - { - int32 connectionId = workerTask->connectionId; - - ResultStatus status = MultiClientResultStatus(connectionId); - if (status == CLIENT_RESULT_BUSY) - { - MultiClientCancel(connectionId); - } - } - - /* give the backend some time to flush its response */ - workerTask->taskStatus = TASK_CANCELED; - break; - } - - case TASK_CANCELED: - { - if (workerTask->connectionId != INVALID_CONNECTION_ID) - { - MultiClientDisconnect(workerTask->connectionId); - workerTask->connectionId = INVALID_CONNECTION_ID; - } - - if (workerTask->taskId == JOB_CLEANUP_TASK_ID) - { - StringInfo jobDirectoryName = JobDirectoryName(workerTask->jobId); - CitusRemoveDirectory(jobDirectoryName->data); - } - - workerTask->taskStatus = TASK_TO_REMOVE; - break; - } - - case TASK_TO_REMOVE: - default: - { - /* we fatal here to avoid leaking client-side resources */ - ereport(FATAL, (errmsg("invalid task status: %d", workerTask->taskStatus))); - break; - } - } - - Assert(workerTask->failureCount <= MAX_TASK_FAILURE_COUNT); -} - - -/* - * MarkWorkerTaskAsFailed marks the given worker task as failed - * and increases the failure count. Failure count is used to - * determine if the task should be marked as permanently failed. - */ -static void -MarkWorkerTaskAsFailed(WorkerTask *workerTask) -{ - workerTask->taskStatus = TASK_FAILED; - workerTask->failureCount++; -} - - -/* Wrapper function to remove the worker task from the shared hash. */ -static void -RemoveWorkerTask(WorkerTask *workerTask, HTAB *WorkerTasksHash) -{ - void *hashKey = (void *) workerTask; - - WorkerTask *taskRemoved = hash_search(WorkerTasksHash, hashKey, HASH_REMOVE, NULL); - if (taskRemoved == NULL) - { - ereport(FATAL, (errmsg("worker task hash corrupted"))); - } -} - - -/* Wrapper function to create the job directory if it does not already exist. */ -static void -CreateJobDirectoryIfNotExists(uint64 jobId) -{ - StringInfo jobDirectoryName = JobDirectoryName(jobId); - - bool jobDirectoryExists = DirectoryExists(jobDirectoryName); - if (!jobDirectoryExists) - { - CitusCreateDirectory(jobDirectoryName); - } - - FreeStringInfo(jobDirectoryName); -} - - -/* Wrapper function to inititate connection to local backend. */ -static int32 -ConnectToLocalBackend(const char *databaseName, const char *userName) -{ - const char *nodeName = LOCAL_HOST_NAME; - const uint32 nodePort = PostPortNumber; - - /* - * Our client library currently only handles TCP sockets. We therefore do - * not use Unix domain sockets here. - */ - int32 connectionId = MultiClientConnect(nodeName, nodePort, databaseName, userName); - - return connectionId; -} diff --git a/src/backend/distributed/worker/task_tracker_protocol.c b/src/backend/distributed/worker/task_tracker_protocol.c index 0016e96b8..eade29081 100644 --- a/src/backend/distributed/worker/task_tracker_protocol.c +++ b/src/backend/distributed/worker/task_tracker_protocol.c @@ -17,35 +17,6 @@ #include "funcapi.h" #include "miscadmin.h" -#include - -#include "access/htup_details.h" -#include "access/xact.h" -#include "catalog/pg_namespace.h" -#include "catalog/namespace.h" -#include "commands/dbcommands.h" -#include "commands/schemacmds.h" -#include "commands/trigger.h" -#include "distributed/metadata_cache.h" -#include "distributed/multi_client_executor.h" -#include "distributed/multi_server_executor.h" -#include "distributed/resource_lock.h" -#include "distributed/task_tracker.h" -#include "distributed/task_tracker_protocol.h" -#include "distributed/worker_protocol.h" -#include "storage/lwlock.h" -#include "storage/pmsignal.h" -#include "utils/builtins.h" -#include "utils/syscache.h" -#include "utils/lsyscache.h" - - -/* Local functions forward declarations */ -static bool TaskTrackerRunning(void); -static void CreateTask(uint64 jobId, uint32 taskId, char *taskCallString); -static void UpdateTask(WorkerTask *workerTask, char *taskCallString); -static void CleanupTask(WorkerTask *workerTask); - /* exports for SQL callable functions */ PG_FUNCTION_INFO_V1(task_tracker_assign_task); @@ -62,73 +33,9 @@ PG_FUNCTION_INFO_V1(task_tracker_conninfo_cache_invalidate); Datum task_tracker_assign_task(PG_FUNCTION_ARGS) { - uint64 jobId = PG_GETARG_INT64(0); - uint32 taskId = PG_GETARG_UINT32(1); - text *taskCallStringText = PG_GETARG_TEXT_P(2); + ereport(ERROR, (errmsg("This UDF is deprecated."))); - StringInfo jobSchemaName = JobSchemaName(jobId); - - char *taskCallString = text_to_cstring(taskCallStringText); - uint32 taskCallStringLength = strlen(taskCallString); - - - CheckCitusVersion(ERROR); - - /* check that we have a running task tracker on this host */ - bool taskTrackerRunning = TaskTrackerRunning(); - if (!taskTrackerRunning) - { - ereport(ERROR, (errcode(ERRCODE_CANNOT_CONNECT_NOW), - errmsg("the task tracker has been disabled or shut down"))); - } - - /* check that we have enough space in our shared hash for this string */ - if (taskCallStringLength >= MaxTaskStringSize) - { - ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE), - errmsg("task string length (%d) exceeds maximum assignable " - "size (%d)", taskCallStringLength, MaxTaskStringSize), - errhint("Consider increasing citus.max_task_string_size."))); - } - - /* - * If the schema does not exist, we create it. However, the schema does not - * become visible to other processes until the transaction commits, and we - * therefore do not release the resource lock in this case. Otherwise, the - * schema is already visible, and we immediately release the resource lock. - */ - LockJobResource(jobId, AccessExclusiveLock); - bool schemaExists = JobSchemaExists(jobSchemaName); - if (!schemaExists) - { - /* lock gets automatically released upon return from this function */ - CreateJobSchema(jobSchemaName, NULL); - } - else - { - Oid schemaId = get_namespace_oid(jobSchemaName->data, false); - - EnsureSchemaOwner(schemaId); - - UnlockJobResource(jobId, AccessExclusiveLock); - } - - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_EXCLUSIVE); - - /* check if we already have the task in our shared hash */ - WorkerTask *workerTask = WorkerTasksHashFind(jobId, taskId); - if (workerTask == NULL) - { - CreateTask(jobId, taskId, taskCallString); - } - else - { - UpdateTask(workerTask, taskCallString); - } - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); - - PG_RETURN_VOID(); + PG_RETURN_NULL(); } @@ -136,41 +43,9 @@ task_tracker_assign_task(PG_FUNCTION_ARGS) Datum task_tracker_task_status(PG_FUNCTION_ARGS) { - uint64 jobId = PG_GETARG_INT64(0); - uint32 taskId = PG_GETARG_UINT32(1); + ereport(ERROR, (errmsg("This UDF is deprecated."))); - WorkerTask *workerTask = NULL; - uint32 taskStatus = 0; - char *userName = CurrentUserName(); - - CheckCitusVersion(ERROR); - - bool taskTrackerRunning = TaskTrackerRunning(); - - if (taskTrackerRunning) - { - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_SHARED); - - workerTask = WorkerTasksHashFind(jobId, taskId); - if (workerTask == NULL || - (!superuser() && strncmp(userName, workerTask->userName, NAMEDATALEN) != 0)) - { - ereport(ERROR, (errmsg("could not find the worker task"), - errdetail("Task jobId: " UINT64_FORMAT " and taskId: %u", - jobId, taskId))); - } - - taskStatus = (uint32) workerTask->taskStatus; - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); - } - else - { - ereport(ERROR, (errcode(ERRCODE_CANNOT_CONNECT_NOW), - errmsg("the task tracker has been disabled or shut down"))); - } - - PG_RETURN_UINT32(taskStatus); + PG_RETURN_UINT32(0); } @@ -181,62 +56,9 @@ task_tracker_task_status(PG_FUNCTION_ARGS) Datum task_tracker_cleanup_job(PG_FUNCTION_ARGS) { - uint64 jobId = PG_GETARG_INT64(0); + ereport(ERROR, (errmsg("This UDF is deprecated."))); - HASH_SEQ_STATUS status; - - CheckCitusVersion(ERROR); - - StringInfo jobSchemaName = JobSchemaName(jobId); - StringInfo jobDirectoryName = JobDirectoryName(jobId); - - /* - * We'll keep this lock for a while, but that's ok because nothing - * else should be happening on this job. - */ - LockJobResource(jobId, AccessExclusiveLock); - - bool schemaExists = JobSchemaExists(jobSchemaName); - if (schemaExists) - { - Oid schemaId = get_namespace_oid(jobSchemaName->data, false); - - EnsureSchemaOwner(schemaId); - } - - /* - * We first clean up any open connections, and remove tasks belonging to - * this job from the shared hash. - */ - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_EXCLUSIVE); - - hash_seq_init(&status, TaskTrackerTaskHash); - - WorkerTask *currentTask = (WorkerTask *) hash_seq_search(&status); - while (currentTask != NULL) - { - if (currentTask->jobId == jobId) - { - CleanupTask(currentTask); - } - - currentTask = (WorkerTask *) hash_seq_search(&status); - } - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); - - /* - * We then delete the job directory and schema, if they exist. This cleans - * up all intermediate files and tables allocated for the job. Note that the - * schema drop call can block if another process is creating the schema or - * writing to a table within the schema. - */ - CitusRemoveDirectory(jobDirectoryName->data); - - RemoveJobSchema(jobSchemaName); - UnlockJobResource(jobId, AccessExclusiveLock); - - PG_RETURN_VOID(); + PG_RETURN_NULL(); } @@ -250,211 +72,5 @@ task_tracker_cleanup_job(PG_FUNCTION_ARGS) Datum task_tracker_conninfo_cache_invalidate(PG_FUNCTION_ARGS) { - if (!CALLED_AS_TRIGGER(fcinfo)) - { - ereport(ERROR, (errcode(ERRCODE_E_R_I_E_TRIGGER_PROTOCOL_VIOLATED), - errmsg("must be called as trigger"))); - } - - CheckCitusVersion(ERROR); - - /* no-op in community edition */ - PG_RETURN_DATUM(PointerGetDatum(NULL)); } - - -/* - * TaskTrackerRunning checks if the task tracker process is running. To do this, - * the function checks if the task tracker is configured to start up, and infers - * from shared memory that the tracker hasn't received a shut down request. - */ -static bool -TaskTrackerRunning(void) -{ - bool taskTrackerRunning = true; - - /* if postmaster shut down, infer task tracker shut down from it */ - bool postmasterAlive = PostmasterIsAlive(); - if (!postmasterAlive) - { - return false; - } - - /* - * When the task tracker receives a termination signal, it inserts a special - * marker task to the shared hash. We need to look up this marker task since - * the postmaster doesn't send a terminate signal to running backends. - */ - LWLockAcquire(&WorkerTasksSharedState->taskHashLock, LW_SHARED); - - WorkerTask *workerTask = WorkerTasksHashFind(RESERVED_JOB_ID, - SHUTDOWN_MARKER_TASK_ID); - if (workerTask != NULL) - { - taskTrackerRunning = false; - } - - LWLockRelease(&WorkerTasksSharedState->taskHashLock); - - return taskTrackerRunning; -} - - -/* - * CreateJobSchema creates a job schema with the given schema name. Note that - * this function ensures that our pg_ prefixed schema names can be created. - * Further note that the created schema does not become visible to other - * processes until the transaction commits. - * - * If schemaOwner is NULL, then current user is used. - */ -void -CreateJobSchema(StringInfo schemaName, char *schemaOwner) -{ - const char *queryString = NULL; - - Oid savedUserId = InvalidOid; - int savedSecurityContext = 0; - RoleSpec currentUserRole = { 0 }; - - /* allow schema names that start with pg_ */ - bool oldAllowSystemTableMods = allowSystemTableMods; - allowSystemTableMods = true; - - /* ensure we're allowed to create this schema */ - GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); - SetUserIdAndSecContext(CitusExtensionOwner(), SECURITY_LOCAL_USERID_CHANGE); - - if (schemaOwner == NULL) - { - schemaOwner = GetUserNameFromId(savedUserId, false); - } - - /* build a CREATE SCHEMA statement */ - currentUserRole.type = T_RoleSpec; - currentUserRole.roletype = ROLESPEC_CSTRING; - currentUserRole.rolename = schemaOwner; - currentUserRole.location = -1; - - CreateSchemaStmt *createSchemaStmt = makeNode(CreateSchemaStmt); - createSchemaStmt->schemaname = schemaName->data; - createSchemaStmt->schemaElts = NIL; - - /* actually create schema with the current user as owner */ - createSchemaStmt->authrole = ¤tUserRole; - CreateSchemaCommand(createSchemaStmt, queryString, -1, -1); - - CommandCounterIncrement(); - - /* and reset environment */ - SetUserIdAndSecContext(savedUserId, savedSecurityContext); - allowSystemTableMods = oldAllowSystemTableMods; -} - - -/* - * CreateTask creates a new task in shared hash, initializes the task, and sets - * the task to assigned state. Note that this function expects the caller to - * hold an exclusive lock over the shared hash. - */ -static void -CreateTask(uint64 jobId, uint32 taskId, char *taskCallString) -{ - const char *databaseName = CurrentDatabaseName(); - char *userName = CurrentUserName(); - - /* increase task priority for cleanup tasks */ - uint32 assignmentTime = (uint32) time(NULL); - if (taskId == JOB_CLEANUP_TASK_ID) - { - assignmentTime = HIGH_PRIORITY_TASK_TIME; - } - - /* enter the worker task into shared hash and initialize the task */ - WorkerTask *workerTask = WorkerTasksHashEnter(jobId, taskId); - workerTask->assignedAt = assignmentTime; - strlcpy(workerTask->taskCallString, taskCallString, MaxTaskStringSize); - - workerTask->taskStatus = TASK_ASSIGNED; - workerTask->connectionId = INVALID_CONNECTION_ID; - workerTask->failureCount = 0; - strlcpy(workerTask->databaseName, databaseName, NAMEDATALEN); - strlcpy(workerTask->userName, userName, NAMEDATALEN); -} - - -/* - * UpdateTask updates the call string text for an already existing task. Note - * that this function expects the caller to hold an exclusive lock over the - * shared hash. - */ -static void -UpdateTask(WorkerTask *workerTask, char *taskCallString) -{ - TaskStatus taskStatus = workerTask->taskStatus; - Assert(taskStatus != TASK_STATUS_INVALID_FIRST); - - /* - * 1. If the task has succeeded or has been canceled, we don't do anything. - * 2. If the task has permanently failed, we update the task call string, - * reset the failure count, and change the task's status to schedulable. - * 3. If the task is in conduit, we update the task call string, and reset - * the failure count. - */ - if (taskStatus == TASK_SUCCEEDED || taskStatus == TASK_CANCEL_REQUESTED || - taskStatus == TASK_CANCELED) - { - /* nothing to do */ - } - else if (taskStatus == TASK_PERMANENTLY_FAILED) - { - strlcpy(workerTask->taskCallString, taskCallString, MaxTaskStringSize); - workerTask->failureCount = 0; - workerTask->taskStatus = TASK_ASSIGNED; - } - else - { - strlcpy(workerTask->taskCallString, taskCallString, MaxTaskStringSize); - workerTask->failureCount = 0; - } -} - - -/* Cleans up connection and shared hash entry associated with the given task. */ -static void -CleanupTask(WorkerTask *workerTask) -{ - void *hashKey = (void *) workerTask; - - /* - * If the connection is still valid, the master node decided to terminate - * the task prematurely. This can happen when the user wants to cancel the - * query, or when a speculatively executed task finishes elsewhere and the - * query completes. - */ - if (workerTask->connectionId != INVALID_CONNECTION_ID) - { - /* - * The task tracker process owns the connections to local backends, and - * we cannot interefere with those connections from another process. We - * therefore ask the task tracker to clean up the connection and to - * remove the task from the shared hash. Note that one of the cleaned up - * tasks will always be the clean-up task itself. - */ - ereport(DEBUG3, (errmsg("requesting cancel for worker task"), - errdetail("Task jobId: " UINT64_FORMAT " and taskId: %u", - workerTask->jobId, workerTask->taskId))); - - workerTask->taskStatus = TASK_CANCEL_REQUESTED; - return; - } - - /* remove task from the shared hash */ - WorkerTask *taskRemoved = hash_search(TaskTrackerTaskHash, hashKey, HASH_REMOVE, - NULL); - if (taskRemoved == NULL) - { - ereport(FATAL, (errmsg("worker task hash corrupted"))); - } -} diff --git a/src/backend/distributed/worker/worker_data_fetch_protocol.c b/src/backend/distributed/worker/worker_data_fetch_protocol.c index f689c975d..329c8dd5f 100644 --- a/src/backend/distributed/worker/worker_data_fetch_protocol.c +++ b/src/backend/distributed/worker/worker_data_fetch_protocol.c @@ -41,7 +41,7 @@ #include "distributed/relay_utility.h" #include "distributed/remote_commands.h" #include "distributed/resource_lock.h" -#include "distributed/task_tracker.h" + #include "distributed/worker_protocol.h" #include "distributed/version_compat.h" #include "nodes/makefuncs.h" diff --git a/src/backend/distributed/worker/worker_merge_protocol.c b/src/backend/distributed/worker/worker_merge_protocol.c index d80b075d1..2ca6552b1 100644 --- a/src/backend/distributed/worker/worker_merge_protocol.c +++ b/src/backend/distributed/worker/worker_merge_protocol.c @@ -34,8 +34,7 @@ #include "distributed/metadata_cache.h" #include "distributed/worker_protocol.h" #include "distributed/version_compat.h" -#include "distributed/task_tracker_protocol.h" -#include "distributed/task_tracker.h" + #include "executor/spi.h" #include "nodes/makefuncs.h" #include "parser/parse_type.h" @@ -54,6 +53,7 @@ static void CreateTaskTable(StringInfo schemaName, StringInfo relationName, List *columnNameList, List *columnTypeList); static void CopyTaskFilesFromDirectory(StringInfo schemaName, StringInfo relationName, StringInfo sourceDirectoryName, Oid userId); +static void CreateJobSchema(StringInfo schemaName, char *schemaOwner); /* exports for SQL callable functions */ @@ -88,6 +88,58 @@ worker_create_schema(PG_FUNCTION_ARGS) } +/* + * CreateJobSchema creates a job schema with the given schema name. Note that + * this function ensures that our pg_ prefixed schema names can be created. + * Further note that the created schema does not become visible to other + * processes until the transaction commits. + * + * If schemaOwner is NULL, then current user is used. + */ +static void +CreateJobSchema(StringInfo schemaName, char *schemaOwner) +{ + const char *queryString = NULL; + + Oid savedUserId = InvalidOid; + int savedSecurityContext = 0; + RoleSpec currentUserRole = { 0 }; + + /* allow schema names that start with pg_ */ + bool oldAllowSystemTableMods = allowSystemTableMods; + allowSystemTableMods = true; + + /* ensure we're allowed to create this schema */ + GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); + SetUserIdAndSecContext(CitusExtensionOwner(), SECURITY_LOCAL_USERID_CHANGE); + + if (schemaOwner == NULL) + { + schemaOwner = GetUserNameFromId(savedUserId, false); + } + + /* build a CREATE SCHEMA statement */ + currentUserRole.type = T_RoleSpec; + currentUserRole.roletype = ROLESPEC_CSTRING; + currentUserRole.rolename = schemaOwner; + currentUserRole.location = -1; + + CreateSchemaStmt *createSchemaStmt = makeNode(CreateSchemaStmt); + createSchemaStmt->schemaname = schemaName->data; + createSchemaStmt->schemaElts = NIL; + + /* actually create schema with the current user as owner */ + createSchemaStmt->authrole = ¤tUserRole; + CreateSchemaCommand(createSchemaStmt, queryString, -1, -1); + + CommandCounterIncrement(); + + /* and reset environment */ + SetUserIdAndSecContext(savedUserId, savedSecurityContext); + allowSystemTableMods = oldAllowSystemTableMods; +} + + /* * worker_repartition_cleanup removes the job directory and schema with the given job id . */ @@ -208,91 +260,7 @@ worker_merge_files_into_table(PG_FUNCTION_ARGS) Datum worker_merge_files_and_run_query(PG_FUNCTION_ARGS) { - uint64 jobId = PG_GETARG_INT64(0); - uint32 taskId = PG_GETARG_UINT32(1); - text *createMergeTableQueryText = PG_GETARG_TEXT_P(2); - text *createIntermediateTableQueryText = PG_GETARG_TEXT_P(3); - - const char *createMergeTableQuery = text_to_cstring(createMergeTableQueryText); - const char *createIntermediateTableQuery = - text_to_cstring(createIntermediateTableQueryText); - - StringInfo taskDirectoryName = TaskDirectoryName(jobId, taskId); - StringInfo jobSchemaName = JobSchemaName(jobId); - StringInfo intermediateTableName = TaskTableName(taskId); - StringInfo mergeTableName = makeStringInfo(); - StringInfo setSearchPathString = makeStringInfo(); - Oid savedUserId = InvalidOid; - int savedSecurityContext = 0; - Oid userId = GetUserId(); - - CheckCitusVersion(ERROR); - - /* - * If the schema for the job isn't already created by the task tracker - * protocol, we fall to using the default 'public' schema. - */ - bool schemaExists = JobSchemaExists(jobSchemaName); - if (!schemaExists) - { - resetStringInfo(jobSchemaName); - appendStringInfoString(jobSchemaName, "public"); - } - else - { - Oid schemaId = get_namespace_oid(jobSchemaName->data, false); - - EnsureSchemaOwner(schemaId); - } - - appendStringInfo(setSearchPathString, SET_SEARCH_PATH_COMMAND, jobSchemaName->data); - - /* Add "public" to search path to access UDFs in public schema */ - appendStringInfo(setSearchPathString, ",public"); - - int connected = SPI_connect(); - if (connected != SPI_OK_CONNECT) - { - ereport(ERROR, (errmsg("could not connect to SPI manager"))); - } - - int setSearchPathResult = SPI_exec(setSearchPathString->data, 0); - if (setSearchPathResult < 0) - { - ereport(ERROR, (errmsg("execution was not successful \"%s\"", - setSearchPathString->data))); - } - - int createMergeTableResult = SPI_exec(createMergeTableQuery, 0); - if (createMergeTableResult < 0) - { - ereport(ERROR, (errmsg("execution was not successful \"%s\"", - createMergeTableQuery))); - } - - /* need superuser to copy from files */ - GetUserIdAndSecContext(&savedUserId, &savedSecurityContext); - SetUserIdAndSecContext(CitusExtensionOwner(), SECURITY_LOCAL_USERID_CHANGE); - - appendStringInfo(mergeTableName, "%s%s", intermediateTableName->data, - MERGE_TABLE_SUFFIX); - CopyTaskFilesFromDirectory(jobSchemaName, mergeTableName, taskDirectoryName, - userId); - - SetUserIdAndSecContext(savedUserId, savedSecurityContext); - - int createIntermediateTableResult = SPI_exec(createIntermediateTableQuery, 0); - if (createIntermediateTableResult < 0) - { - ereport(ERROR, (errmsg("execution was not successful \"%s\"", - createIntermediateTableQuery))); - } - - int finished = SPI_finish(); - if (finished != SPI_OK_FINISH) - { - ereport(ERROR, (errmsg("could not disconnect from SPI manager"))); - } + ereport(ERROR, (errmsg("This UDF is deprecated."))); PG_RETURN_VOID(); } diff --git a/src/backend/distributed/worker/worker_partition_protocol.c b/src/backend/distributed/worker/worker_partition_protocol.c index 4d1d5aa14..95854932c 100644 --- a/src/backend/distributed/worker/worker_partition_protocol.c +++ b/src/backend/distributed/worker/worker_partition_protocol.c @@ -499,27 +499,6 @@ ClosePartitionFiles(FileOutputStream *partitionFileArray, uint32 fileCount) } -/* - * MasterJobDirectoryName constructs a standardized job - * directory path for the given job id on the master node. - */ -StringInfo -MasterJobDirectoryName(uint64 jobId) -{ - StringInfo jobDirectoryName = makeStringInfo(); - - /* - * We use the default tablespace in {datadir}/base. Further, we need to - * apply padding on our 64-bit job id, and hence can't use UINT64_FORMAT. - */ - appendStringInfo(jobDirectoryName, "base/%s/%s%0*" INT64_MODIFIER "u", - PG_JOB_CACHE_DIR, MASTER_JOB_DIRECTORY_PREFIX, - MIN_JOB_DIRNAME_WIDTH, jobId); - - return jobDirectoryName; -} - - /* * JobDirectoryName Constructs a standardized job * directory path for the given job id on the worker nodes. @@ -813,6 +792,25 @@ CitusRemoveDirectory(const char *filename) } +/* + * RepartitionCleanupJobDirectories cleans up all files in the job cache directory + * as part of this process's start-up logic. The files could be leaked from + * repartition joins. + */ +void +RepartitionCleanupJobDirectories(void) +{ + /* use the default tablespace in {datadir}/base */ + StringInfo jobCacheDirectory = makeStringInfo(); + appendStringInfo(jobCacheDirectory, "base/%s", PG_JOB_CACHE_DIR); + + CitusRemoveDirectory(jobCacheDirectory->data); + CitusCreateDirectory(jobCacheDirectory); + + FreeStringInfo(jobCacheDirectory); +} + + /* Moves directory from old path to the new one. */ static void RenameDirectory(StringInfo oldDirectoryName, StringInfo newDirectoryName) diff --git a/src/backend/distributed/worker/worker_sql_task_protocol.c b/src/backend/distributed/worker/worker_sql_task_protocol.c index 46ef8f147..8cbac012a 100644 --- a/src/backend/distributed/worker/worker_sql_task_protocol.c +++ b/src/backend/distributed/worker/worker_sql_task_protocol.c @@ -75,47 +75,9 @@ PG_FUNCTION_INFO_V1(worker_execute_sql_task); Datum worker_execute_sql_task(PG_FUNCTION_ARGS) { - uint64 jobId = PG_GETARG_INT64(0); - uint32 taskId = PG_GETARG_UINT32(1); - text *queryText = PG_GETARG_TEXT_P(2); - char *queryString = text_to_cstring(queryText); - bool binaryCopyFormat = PG_GETARG_BOOL(3); + ereport(ERROR, (errmsg("This UDF is deprecated."))); - - /* job directory is created prior to scheduling the task */ - StringInfo jobDirectoryName = JobDirectoryName(jobId); - StringInfo taskFilename = UserTaskFilename(jobDirectoryName, taskId); - - Query *query = ParseQueryString(queryString, NULL, 0); - int64 tuplesSent = WorkerExecuteSqlTask(query, taskFilename->data, binaryCopyFormat); - - PG_RETURN_INT64(tuplesSent); -} - - -/* - * WorkerExecuteSqlTask executes an already-parsed query and writes the result - * to the given task file. - */ -int64 -WorkerExecuteSqlTask(Query *query, char *taskFilename, bool binaryCopyFormat) -{ - ParamListInfo paramListInfo = NULL; - - EState *estate = CreateExecutorState(); - MemoryContext tupleContext = GetPerTupleMemoryContext(estate); - TaskFileDestReceiver *taskFileDest = - (TaskFileDestReceiver *) CreateFileDestReceiver(taskFilename, tupleContext, - binaryCopyFormat); - - ExecuteQueryIntoDestReceiver(query, paramListInfo, (DestReceiver *) taskFileDest); - - int64 tuplesSent = taskFileDest->tuplesSent; - - taskFileDest->pub.rDestroy((DestReceiver *) taskFileDest); - FreeExecutorState(estate); - - return tuplesSent; + PG_RETURN_INT64(0); } diff --git a/src/include/distributed/citus_custom_scan.h b/src/include/distributed/citus_custom_scan.h index dcddb27a3..e6d5389f7 100644 --- a/src/include/distributed/citus_custom_scan.h +++ b/src/include/distributed/citus_custom_scan.h @@ -31,7 +31,6 @@ typedef struct CitusScanState /* custom scan methods for all executors */ extern CustomScanMethods AdaptiveExecutorCustomScanMethods; -extern CustomScanMethods TaskTrackerCustomScanMethods; extern CustomScanMethods NonPushableInsertSelectCustomScanMethods; extern CustomScanMethods DelayedErrorCustomScanMethods; diff --git a/src/include/distributed/citus_nodefuncs.h b/src/include/distributed/citus_nodefuncs.h index d25f6222c..9dbafeb6f 100644 --- a/src/include/distributed/citus_nodefuncs.h +++ b/src/include/distributed/citus_nodefuncs.h @@ -49,7 +49,6 @@ extern void OutRelationShard(OUTFUNC_ARGS); extern void OutRelationRowLock(OUTFUNC_ARGS); extern void OutTask(OUTFUNC_ARGS); extern void OutLocalPlannedStatement(OUTFUNC_ARGS); -extern void OutTaskExecution(OUTFUNC_ARGS); extern void OutDeferredErrorMessage(OUTFUNC_ARGS); extern void OutGroupShardPlacement(OUTFUNC_ARGS); @@ -77,7 +76,6 @@ extern void CopyNodeRelationRowLock(COPYFUNC_ARGS); extern void CopyNodeTask(COPYFUNC_ARGS); extern void CopyNodeLocalPlannedStatement(COPYFUNC_ARGS); extern void CopyNodeTaskQuery(COPYFUNC_ARGS); -extern void CopyNodeTaskExecution(COPYFUNC_ARGS); extern void CopyNodeDeferredErrorMessage(COPYFUNC_ARGS); #endif /* CITUS_NODEFUNCS_H */ diff --git a/src/include/distributed/citus_nodes.h b/src/include/distributed/citus_nodes.h index c593f48f5..fe244082c 100644 --- a/src/include/distributed/citus_nodes.h +++ b/src/include/distributed/citus_nodes.h @@ -60,7 +60,6 @@ typedef enum CitusNodeTag T_UsedDistributedSubPlan, T_Task, T_LocalPlannedStatement, - T_TaskExecution, T_ShardInterval, T_ShardPlacement, T_RelationShard, diff --git a/src/include/distributed/intermediate_result_pruning.h b/src/include/distributed/intermediate_result_pruning.h index 5bde93313..d207397c2 100644 --- a/src/include/distributed/intermediate_result_pruning.h +++ b/src/include/distributed/intermediate_result_pruning.h @@ -17,6 +17,7 @@ * UINT32_MAX is reserved in pg_dist_node, so we can use it safely. */ #define LOCAL_NODE_ID UINT32_MAX +#define LOCAL_HOST_NAME "localhost" /* connect to local backends using this name */ extern bool LogIntermediateResults; diff --git a/src/include/distributed/multi_client_executor.h b/src/include/distributed/multi_client_executor.h index 764b168cb..529929aef 100644 --- a/src/include/distributed/multi_client_executor.h +++ b/src/include/distributed/multi_client_executor.h @@ -78,18 +78,6 @@ typedef enum CLIENT_BATCH_QUERY_DONE = 3 } BatchQueryStatus; - -/* Enumeration to track whether a task is ready to run and, if not, what it's blocked on*/ -typedef enum TaskExecutionStatus -{ - TASK_STATUS_INVALID = 0, - TASK_STATUS_ERROR, /* error occured */ - TASK_STATUS_READY, /* task ready to be processed further */ - TASK_STATUS_SOCKET_READ, /* waiting for connection to become ready for reads */ - TASK_STATUS_SOCKET_WRITE /* waiting for connection to become ready for writes */ -} TaskExecutionStatus; - - struct pollfd; /* forward declared, to avoid having to include poll.h */ typedef struct WaitInfo diff --git a/src/include/distributed/multi_executor.h b/src/include/distributed/multi_executor.h index 675734db9..a7d68b4e1 100644 --- a/src/include/distributed/multi_executor.h +++ b/src/include/distributed/multi_executor.h @@ -124,7 +124,6 @@ extern uint64 ExecuteTaskListIntoTupleDest(RowModifyLevel modLevel, List *taskLi extern bool IsCitusCustomState(PlanState *planState); extern TupleTableSlot * CitusExecScan(CustomScanState *node); extern TupleTableSlot * ReturnTupleFromTuplestore(CitusScanState *scanState); -extern void LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob); extern void ReadFileIntoTupleStore(char *fileName, char *copyFormat, TupleDesc tupleDescriptor, Tuplestorestate *tupstore); extern Query * ParseQueryString(const char *queryString, Oid *paramOids, int numParams); diff --git a/src/include/distributed/multi_logical_planner.h b/src/include/distributed/multi_logical_planner.h index c8c9e7a9f..ae6224d47 100644 --- a/src/include/distributed/multi_logical_planner.h +++ b/src/include/distributed/multi_logical_planner.h @@ -188,7 +188,6 @@ extern MultiTreeRoot * MultiLogicalPlanCreate(Query *originalQuery, Query *query PlannerRestrictionContext * plannerRestrictionContext); extern bool FindNodeCheck(Node *node, bool (*check)(Node *)); -extern bool SingleRelationRepartitionSubquery(Query *queryTree); extern bool TargetListOnPartitionColumn(Query *query, List *targetEntryList); extern bool FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *)); extern bool IsCitusTableRTE(Node *node); diff --git a/src/include/distributed/multi_physical_planner.h b/src/include/distributed/multi_physical_planner.h index cb7f160d2..ed531c840 100644 --- a/src/include/distributed/multi_physical_planner.h +++ b/src/include/distributed/multi_physical_planner.h @@ -45,9 +45,8 @@ (" UINT64_FORMAT ", %d, %s, '%s', '%s'::regtype, %s)" #define MERGE_FILES_INTO_TABLE_COMMAND "SELECT worker_merge_files_into_table \ (" UINT64_FORMAT ", %d, '%s', '%s')" -#define MERGE_FILES_AND_RUN_QUERY_COMMAND \ - "SELECT worker_merge_files_and_run_query(" UINT64_FORMAT ", %d, %s, %s)" +extern int RepartitionJoinBucketCountPerNode; typedef enum CitusRTEKind { @@ -181,28 +180,6 @@ typedef struct MapMergeJob List *mergeTaskList; } MapMergeJob; - -/* - * Task represents an executable unit of work. We conceptualize our tasks into - * compute and data fetch task types. SQL, map, and merge tasks are considered - * as compute tasks; and map fetch, and merge fetch tasks are data - * fetch tasks. We also forward declare the task execution struct here to avoid - * including the executor header files. - * - * We currently do not take replication model into account for tasks other - * than modifications. When it is set to REPLICATION_MODEL_2PC, the execution - * of the modification task is done with two-phase commit. Set it to - * REPLICATION_MODEL_INVALID if it is not relevant for the task. - * - * NB: Changing this requires also changing _outTask in citus_outfuncs and _readTask - * in citus_readfuncs to correctly (de)serialize this struct. - * - * INSERT ... SELECT queries and modify queries with subqueries or multiple tables - * set modifyWithSubquery to true. We need to use it to take the necessary locks - * to get consistent results for subqueries. - */ -typedef struct TaskExecution TaskExecution; - typedef enum TaskQueryType { TASK_QUERY_NULL, @@ -286,7 +263,6 @@ typedef struct Task uint32 upstreamTaskId; /* only applies to data fetch tasks */ ShardInterval *shardInterval; /* only applies to merge tasks */ bool assignmentConstrained; /* only applies to merge tasks */ - TaskExecution *taskExecution; /* used by task tracker executor */ char replicationModel; /* only applies to modify tasks */ /* diff --git a/src/include/distributed/multi_resowner.h b/src/include/distributed/multi_resowner.h deleted file mode 100644 index fdacece4e..000000000 --- a/src/include/distributed/multi_resowner.h +++ /dev/null @@ -1,21 +0,0 @@ -/*------------------------------------------------------------------------- - * - * multi_resowner.h - * Citus resource owner integration. - * - * Copyright (c) Citus Data, Inc. - *------------------------------------------------------------------------- - */ -#ifndef MULTI_RESOWNER_H -#define MULTI_RESOWNER_H - -#include "utils/resowner.h" - -/* resowner functions for temporary job directory management */ -extern void ResourceOwnerEnlargeJobDirectories(ResourceOwner owner); -extern void ResourceOwnerRememberJobDirectory(ResourceOwner owner, - uint64 jobId); -extern void ResourceOwnerForgetJobDirectory(ResourceOwner owner, - uint64 jobId); - -#endif /* MULTI_RESOWNER_H */ diff --git a/src/include/distributed/multi_server_executor.h b/src/include/distributed/multi_server_executor.h index fc46747a5..52b260880 100644 --- a/src/include/distributed/multi_server_executor.h +++ b/src/include/distributed/multi_server_executor.h @@ -15,27 +15,9 @@ #define MULTI_SERVER_EXECUTOR_H #include "distributed/multi_physical_planner.h" -#include "distributed/task_tracker.h" + #include "distributed/worker_manager.h" - -#define MAX_TASK_EXECUTION_FAILURES 3 /* allowed failure count for one task */ -#define MAX_TRACKER_FAILURE_COUNT 3 /* allowed failure count for one tracker */ -#define RESERVED_FD_COUNT 64 /* file descriptors unavailable to executor */ - -/* copy out query results */ -#define EXECUTE_SQL_TASK_TO_FILE_BINARY \ - "SELECT worker_execute_sql_task("UINT64_FORMAT ", %u, %s, true)" -#define EXECUTE_SQL_TASK_TO_FILE_TEXT \ - "SELECT worker_execute_sql_task("UINT64_FORMAT ", %u, %s, false)" - -/* Task tracker executor related defines */ -#define TASK_ASSIGNMENT_QUERY "SELECT task_tracker_assign_task \ - ("UINT64_FORMAT ", %u, %s);" -#define TASK_STATUS_QUERY "SELECT task_tracker_task_status("UINT64_FORMAT ", %u);" -#define JOB_CLEANUP_QUERY "SELECT task_tracker_cleanup_job("UINT64_FORMAT ")" -#define JOB_CLEANUP_TASK_ID INT_MAX - /* Adaptive executor repartioning related defines */ #define WORKER_CREATE_SCHEMA_QUERY "SELECT worker_create_schema (" UINT64_FORMAT ", %s);" #define WORKER_REPARTITION_CLEANUP_QUERY "SELECT worker_repartition_cleanup (" \ @@ -43,52 +25,12 @@ ");" -/* Enumeration to track one task's execution status */ -typedef enum -{ - /* used for task tracker executor */ - EXEC_TASK_INVALID_FIRST = 0, - EXEC_TASK_DONE = 1, - EXEC_TASK_UNASSIGNED = 2, - EXEC_TASK_QUEUED = 3, - EXEC_TASK_TRACKER_RETRY = 4, - EXEC_TASK_TRACKER_FAILED = 5, - EXEC_SOURCE_TASK_TRACKER_RETRY = 6, - EXEC_SOURCE_TASK_TRACKER_FAILED = 7, -} TaskExecStatus; - - -/* Enumeration to track file transmits to the coordinator node */ -typedef enum -{ - EXEC_TRANSMIT_INVALID_FIRST = 0, - EXEC_TRANSMIT_UNASSIGNED = 1, - EXEC_TRANSMIT_QUEUED = 2, - EXEC_TRANSMIT_COPYING = 3, - EXEC_TRANSMIT_TRACKER_RETRY = 4, - EXEC_TRANSMIT_TRACKER_FAILED = 5, - EXEC_TRANSMIT_DONE = 6 -} TransmitExecStatus; - - -/* Enumeration to track a task tracker's connection status */ -typedef enum -{ - TRACKER_STATUS_INVALID_FIRST = 0, - TRACKER_CONNECT_START = 1, - TRACKER_CONNECT_POLL = 2, - TRACKER_CONNECTED = 3, - TRACKER_CONNECTION_FAILED = 4 -} TrackerStatus; - - /* Enumeration that represents distributed executor types */ typedef enum { MULTI_EXECUTOR_INVALID_FIRST = 0, MULTI_EXECUTOR_ADAPTIVE = 1, - MULTI_EXECUTOR_TASK_TRACKER = 2, - MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT = 3 + MULTI_EXECUTOR_NON_PUSHABLE_INSERT_SELECT = 2 } MultiExecutorType; @@ -104,90 +46,16 @@ typedef struct DistributedExecutionStats uint64 totalIntermediateResultSize; } DistributedExecutionStats; - -/* - * TaskExecution holds state that relates to a task's execution for task-tracker - * executor. - */ -struct TaskExecution -{ - CitusNode type; - uint64 jobId; - uint32 taskId; - - TaskExecStatus *taskStatusArray; - TransmitExecStatus *transmitStatusArray; - int32 *connectionIdArray; - int32 *fileDescriptorArray; - uint32 nodeCount; - uint32 currentNodeIndex; - uint32 querySourceNodeIndex; /* only applies to map fetch tasks */ - uint32 failureCount; -}; - - -/* - * TrackerTaskState represents a task's execution status on a particular task - * tracker. This state augments task execution state in that it is associated - * with execution on a particular task tracker. - */ -typedef struct TrackerTaskState -{ - uint64 jobId; - uint32 taskId; - TaskStatus status; - StringInfo taskAssignmentQuery; -} TrackerTaskState; - - -/* - * TaskTracker keeps connection and task related state for a task tracker. The - * task tracker executor then uses this state to open and manage a connection to - * the task tracker; and assign and check status of tasks over this connection. - */ -typedef struct TaskTracker -{ - uint32 workerPort; /* node's port; part of hash table key */ - char workerName[WORKER_LENGTH]; /* node's name; part of hash table key */ - char *userName; /* which user to connect as */ - TrackerStatus trackerStatus; - int32 connectionId; - uint32 connectPollCount; - uint32 connectionFailureCount; - uint32 trackerFailureCount; - - HTAB *taskStateHash; - List *assignedTaskList; - int32 currentTaskIndex; - bool connectionBusy; - TrackerTaskState *connectionBusyOnTask; - List *connectionBusyOnTaskList; -} TaskTracker; - - /* Config variable managed via guc.c */ extern int RemoteTaskCheckInterval; -extern int MaxAssignTaskBatchSize; extern int TaskExecutorType; extern bool EnableRepartitionJoins; -extern bool BinaryMasterCopyFormat; extern int MultiTaskQueryLogLevel; -/* Function declarations for distributed execution */ -extern void MultiTaskTrackerExecute(Job *job); - /* Function declarations common to more than one executor */ extern MultiExecutorType JobExecutorType(DistributedPlan *distributedPlan); -extern void RemoveJobDirectory(uint64 jobId); -extern TaskExecution * InitTaskExecution(Task *task, TaskExecStatus initialStatus); extern bool CheckIfSizeLimitIsExceeded(DistributedExecutionStats *executionStats); -extern void CleanupTaskExecution(TaskExecution *taskExecution); extern void ErrorSizeLimitIsExceeded(void); -extern bool TaskExecutionFailed(TaskExecution *taskExecution); -extern void AdjustStateForFailure(TaskExecution *taskExecution); - - -extern TupleTableSlot * TaskTrackerExecScan(CustomScanState *node); #endif /* MULTI_SERVER_EXECUTOR_H */ diff --git a/src/include/distributed/multi_task_tracker_executor.h b/src/include/distributed/multi_task_tracker_executor.h deleted file mode 100644 index f496c48eb..000000000 --- a/src/include/distributed/multi_task_tracker_executor.h +++ /dev/null @@ -1,8 +0,0 @@ - - -#ifndef MULTI_TASK_TRACKER_EXECUTOR_H -#define MULTI_TASK_TRACKER_EXECUTOR_H - -extern List * CreateTaskListForJobTree(List *jobTaskList, bool viaTaskTracker); - -#endif /* MULTI_TASK_TRACKER_EXECUTOR_H */ diff --git a/src/include/distributed/task_execution_utils.h b/src/include/distributed/task_execution_utils.h new file mode 100644 index 000000000..e7b5a2ee4 --- /dev/null +++ b/src/include/distributed/task_execution_utils.h @@ -0,0 +1,6 @@ +#ifndef TASK_EXECUTION_UTILS_H +#define TASK_EXECUTION_UTILS_H + +extern List * CreateTaskListForJobTree(List *jobTaskList); + +#endif /* TASK_EXECUTION_UTILS_H */ diff --git a/src/include/distributed/task_tracker.h b/src/include/distributed/task_tracker.h deleted file mode 100644 index 3c94606af..000000000 --- a/src/include/distributed/task_tracker.h +++ /dev/null @@ -1,135 +0,0 @@ -/*------------------------------------------------------------------------- - * - * task_tracker.h - * - * Header and type declarations for coordinating execution of tasks and data - * source transfers on worker nodes. - * - * Copyright (c) Citus Data, Inc. - * - * $Id$ - * - *------------------------------------------------------------------------- - */ - -#ifndef TASK_TRACKER_H -#define TASK_TRACKER_H - -#include "storage/lwlock.h" -#include "utils/hsearch.h" - - -#define HIGH_PRIORITY_TASK_TIME 1 /* assignment time for high priority tasks */ -#define RESERVED_JOB_ID 1 /* reserved for cleanup and shutdown tasks */ -#define SHUTDOWN_MARKER_TASK_ID UINT_MAX /* used to identify task tracker shutdown */ -#define MAX_TASK_FAILURE_COUNT 2 /* allowed failure count for one task */ -#define LOCAL_HOST_NAME "localhost" /* connect to local backends using this name */ -#define TASK_CALL_STRING_SIZE 12288 /* max length of task call string */ -#define TEMPLATE0_NAME "template0" /* skip job schema cleanup for template0 */ -#define JOB_SCHEMA_CLEANUP "SELECT worker_cleanup_job_schema_cache()" - - -/* - * TaskStatus represents execution status of worker tasks. The assigned and - * cancel requested statuses are set by the master node; all other statuses are - * assigned by the task tracker as the worker task makes progress. - */ -typedef enum -{ - TASK_STATUS_INVALID_FIRST = 0, - TASK_ASSIGNED = 1, /* master node and task tracker */ - TASK_SCHEDULED = 2, - TASK_RUNNING = 3, - TASK_FAILED = 4, - TASK_PERMANENTLY_FAILED = 5, - TASK_SUCCEEDED = 6, - TASK_CANCEL_REQUESTED = 7, /* master node only */ - TASK_CANCELED = 8, - TASK_TO_REMOVE = 9, - - /* - * The master node's executor uses the following statuses to fully represent - * the execution status of worker tasks, as they are perceived by the master - * node. These statuses in fact don't belong with the task tracker. - */ - TASK_CLIENT_SIDE_QUEUED = 10, - TASK_CLIENT_SIDE_ASSIGN_FAILED = 11, - TASK_CLIENT_SIDE_STATUS_FAILED = 12, - TASK_FILE_TRANSMIT_QUEUED = 13, - TASK_CLIENT_SIDE_TRANSMIT_FAILED = 14, - - /* - * Add new task status types above this comment. Existing types, except for - * TASK_STATUS_LAST, should never have their numbers changed. - */ - TASK_STATUS_LAST -} TaskStatus; - - -/* - * WorkerTask keeps shared memory state for tasks. At a high level, each worker - * task holds onto three different types of state: (a) state assigned by the - * master node, (b) state initialized by the protocol process at task assignment - * time, and (c) state internal to the task tracker process that changes as the - * task make progress. - * - * Since taskCallString is dynamically sized use WORKER_TASK_SIZE instead of - * sizeof(WorkerTask). Use WORKER_TASK_AT to reference an item in WorkerTask array. - */ -typedef struct WorkerTask -{ - uint64 jobId; /* job id (upper 32-bits reserved); part of hash table key */ - uint32 taskId; /* task id; part of hash table key */ - uint32 assignedAt; /* task assignment time in epoch seconds */ - - TaskStatus taskStatus; /* task's current execution status */ - char databaseName[NAMEDATALEN]; /* name to use for local backend connection */ - char userName[NAMEDATALEN]; /* user to use for local backend connection */ - int32 connectionId; /* connection id to local backend */ - uint32 failureCount; /* number of task failures */ - char taskCallString[FLEXIBLE_ARRAY_MEMBER]; /* query or function call string */ -} WorkerTask; - -#define WORKER_TASK_SIZE (offsetof(WorkerTask, taskCallString) + MaxTaskStringSize) - -#define WORKER_TASK_AT(workerTasks, index) \ - ((WorkerTask *) (((char *) (workerTasks)) + (index) * WORKER_TASK_SIZE)) - -/* - * WorkerTasksControlData contains task tracker state shared between - * processes. - */ -typedef struct WorkerTasksSharedStateData -{ - /* Lock protecting workerNodesHash */ - int taskHashTrancheId; - char *taskHashTrancheName; - LWLock taskHashLock; - bool conninfosValid; -} WorkerTasksSharedStateData; - - -extern void TrackerCleanupJobDirectories(void); - -/* Config variables managed via guc.c */ -extern int TaskTrackerDelay; -extern int MaxTrackedTasksPerNode; -extern int MaxRunningTasksPerNode; -extern int MaxTaskStringSize; - -/* State shared by the task tracker and task tracker protocol functions */ -extern WorkerTasksSharedStateData *WorkerTasksSharedState; -extern HTAB *TaskTrackerTaskHash; - -/* Entry point */ -extern void TaskTrackerMain(Datum main_arg); - -/* Function declarations local to the worker module */ -extern WorkerTask * WorkerTasksHashEnter(uint64 jobId, uint32 taskId); -extern WorkerTask * WorkerTasksHashFind(uint64 jobId, uint32 taskId); - -/* Function declarations for starting up and running the task tracker */ -extern void TaskTrackerRegister(void); - - -#endif /* TASK_TRACKER_H */ diff --git a/src/include/distributed/task_tracker_protocol.h b/src/include/distributed/task_tracker_protocol.h deleted file mode 100644 index 7fd82ea94..000000000 --- a/src/include/distributed/task_tracker_protocol.h +++ /dev/null @@ -1,29 +0,0 @@ -/*------------------------------------------------------------------------- - * - * task_tracker_protocol.h - * - * Header and type declarations for assigning tasks to and removing tasks from - * the task tracker running on this node. - * - * Copyright (c) Citus Data, Inc. - * - * $Id$ - * - *------------------------------------------------------------------------- - */ - -#ifndef TASK_TRACKER_PROTOCOL_H -#define TASK_TRACKER_PROTOCOL_H - -#include "fmgr.h" - -extern void CreateJobSchema(StringInfo schemaName, char *schemaOwner); - -/* Function declarations for distributed task management */ -extern Datum task_tracker_assign_task(PG_FUNCTION_ARGS); -extern Datum task_tracker_update_data_fetch_task(PG_FUNCTION_ARGS); -extern Datum task_tracker_task_status(PG_FUNCTION_ARGS); -extern Datum task_tracker_cleanup_job(PG_FUNCTION_ARGS); - - -#endif /* TASK_TRACKER_PROTOCOL_H */ diff --git a/src/include/distributed/worker_protocol.h b/src/include/distributed/worker_protocol.h index 3e1f7c426..7cd88c3b3 100644 --- a/src/include/distributed/worker_protocol.h +++ b/src/include/distributed/worker_protocol.h @@ -110,7 +110,6 @@ extern StringInfo JobSchemaName(uint64 jobId); extern StringInfo TaskTableName(uint32 taskId); extern bool JobSchemaExists(StringInfo schemaName); extern StringInfo JobDirectoryName(uint64 jobId); -extern StringInfo MasterJobDirectoryName(uint64 jobId); extern StringInfo TaskDirectoryName(uint64 jobId, uint32 taskId); extern StringInfo PartitionFilename(StringInfo directoryName, uint32 partitionId); extern bool CacheDirectoryElement(const char *filename); @@ -126,8 +125,7 @@ extern FmgrInfo * GetFunctionInfo(Oid typeId, Oid accessMethodId, int16 procedur extern uint64 ExtractShardIdFromTableName(const char *tableName, bool missingOk); extern List * TableDDLCommandList(const char *nodeName, uint32 nodePort, const char *tableName); -extern int64 WorkerExecuteSqlTask(Query *query, char *taskFilename, - bool binaryCopyFormat); +extern void RepartitionCleanupJobDirectories(void); /* Function declarations shared with the master planner */ diff --git a/src/test/regress/Makefile b/src/test/regress/Makefile index 4f52b688c..4b41d17ac 100644 --- a/src/test/regress/Makefile +++ b/src/test/regress/Makefile @@ -157,11 +157,6 @@ check-multi-mx: all $(pg_regress_multi_check) --load-extension=citus \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_mx_schedule $(EXTRA_TESTS) -check-multi-task-tracker-extra: all - $(pg_regress_multi_check) --load-extension=citus \ - --server-option=citus.task_executor_type=task-tracker \ - -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_task_tracker_extra_schedule $(EXTRA_TESTS) - check-follower-cluster: all $(pg_regress_multi_check) --load-extension=citus --follower-cluster \ -- $(MULTI_REGRESS_OPTS) --schedule=$(citus_abs_srcdir)/multi_follower_schedule $(EXTRA_TESTS) diff --git a/src/test/regress/expected/adaptive_executor.out b/src/test/regress/expected/adaptive_executor.out index d8ebc7d95..73ba772b5 100644 --- a/src/test/regress/expected/adaptive_executor.out +++ b/src/test/regress/expected/adaptive_executor.out @@ -15,7 +15,6 @@ INSERT INTO test VALUES (3,2); -- Set a very high slow start to avoid opening parallel connections SET citus.executor_slow_start_interval TO '60s'; SET citus.max_adaptive_executor_pool_size TO 2; -SET citus.task_executor_type TO 'adaptive'; BEGIN; SELECT count(*) FROM test a JOIN (SELECT x, pg_sleep(0.1) FROM test) b USING (x); count diff --git a/src/test/regress/expected/adaptive_executor_repartition.out b/src/test/regress/expected/adaptive_executor_repartition.out index f89ead3fc..a70e97688 100644 --- a/src/test/regress/expected/adaptive_executor_repartition.out +++ b/src/test/regress/expected/adaptive_executor_repartition.out @@ -1,6 +1,5 @@ CREATE SCHEMA adaptive_executor; SET search_path TO adaptive_executor; -SET citus.task_executor_type to 'adaptive'; SET citus.shard_replication_factor to 1; SET citus.enable_repartition_joins TO true; CREATE TABLE ab(a int, b int); diff --git a/src/test/regress/expected/aggregate_support.out b/src/test/regress/expected/aggregate_support.out index eacf43a32..ae833502e 100644 --- a/src/test/regress/expected/aggregate_support.out +++ b/src/test/regress/expected/aggregate_support.out @@ -387,7 +387,6 @@ select array_agg(val order by valf) from aggdata; (1 row) -- Test TransformSubqueryNode -SET citus.task_executor_type to "task-tracker"; select * FROM ( SELECT key, mode() within group (order by floor(agg1.val/2)) m from aggdata agg1 group by key @@ -404,15 +403,16 @@ select * FROM ( select * FROM ( SELECT key k, avg(distinct floor(agg1.val/2)) m from aggdata agg1 group by key -) subq; +) subq +order by k,m; k | m --------------------------------------------------------------------- 1 | 1 - 5 | - 3 | 2 - 7 | 4 - 6 | 2 | 1.5 + 3 | 2 + 5 | + 6 | + 7 | 4 9 | 0 (7 rows) @@ -426,7 +426,6 @@ select count(*) FROM ( 7 (1 row) -RESET citus.task_executor_type; select key, count(distinct aggdata) from aggdata group by key order by 1, 2; key | count diff --git a/src/test/regress/expected/isolation_append_copy_vs_all.out b/src/test/regress/expected/isolation_append_copy_vs_all.out index 65f8de20f..efea6176c 100644 --- a/src/test/regress/expected/isolation_append_copy_vs_all.out +++ b/src/test/regress/expected/isolation_append_copy_vs_all.out @@ -60,8 +60,8 @@ step s1-initialize: COPY append_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 step s1-begin: BEGIN; step s1-copy: COPY append_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM append_copy AS t1 JOIN append_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM append_copy AS t1 JOIN append_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -399,8 +399,8 @@ create_distributed_table step s1-initialize: COPY append_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM append_copy AS t1 JOIN append_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM append_copy AS t1 JOIN append_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data diff --git a/src/test/regress/expected/isolation_hash_copy_vs_all.out b/src/test/regress/expected/isolation_hash_copy_vs_all.out index b3a90ec0b..6a47d6688 100644 --- a/src/test/regress/expected/isolation_hash_copy_vs_all.out +++ b/src/test/regress/expected/isolation_hash_copy_vs_all.out @@ -60,8 +60,8 @@ step s1-initialize: COPY hash_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && step s1-begin: BEGIN; step s1-copy: COPY hash_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -356,10 +356,10 @@ create_distributed_table step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table @@ -380,10 +380,10 @@ create_distributed_table step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table @@ -404,10 +404,10 @@ create_distributed_table step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table @@ -428,10 +428,10 @@ create_distributed_table step s1-recreate-with-replication-2: - DROP TABLE hash_copy; - SET citus.shard_replication_factor TO 2; - CREATE TABLE hash_copy(id integer, data text, int_data int); - SELECT create_distributed_table('hash_copy', 'id'); + DROP TABLE hash_copy; + SET citus.shard_replication_factor TO 2; + CREATE TABLE hash_copy(id integer, data text, int_data int); + SELECT create_distributed_table('hash_copy', 'id'); create_distributed_table @@ -492,8 +492,8 @@ create_distributed_table step s1-initialize: COPY hash_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data diff --git a/src/test/regress/expected/isolation_partitioned_copy_vs_all.out b/src/test/regress/expected/isolation_partitioned_copy_vs_all.out index 3b578a581..961152e0f 100644 --- a/src/test/regress/expected/isolation_partitioned_copy_vs_all.out +++ b/src/test/regress/expected/isolation_partitioned_copy_vs_all.out @@ -60,8 +60,8 @@ step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, step s1-begin: BEGIN; step s1-copy: COPY partitioned_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -335,8 +335,8 @@ create_distributed_table step s1-initialize: COPY partitioned_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data diff --git a/src/test/regress/expected/isolation_range_copy_vs_all.out b/src/test/regress/expected/isolation_range_copy_vs_all.out index 5e0e7e474..978fda21b 100644 --- a/src/test/regress/expected/isolation_range_copy_vs_all.out +++ b/src/test/regress/expected/isolation_range_copy_vs_all.out @@ -60,8 +60,8 @@ step s1-initialize: COPY range_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 & step s1-begin: BEGIN; step s1-copy: COPY range_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM range_copy AS t1 JOIN range_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM range_copy AS t1 JOIN range_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -413,8 +413,8 @@ create_distributed_table step s1-initialize: COPY range_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM range_copy AS t1 JOIN range_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SET citus.enable_repartition_joins TO ON; + SELECT * FROM range_copy AS t1 JOIN range_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data diff --git a/src/test/regress/expected/isolation_reference_copy_vs_all.out b/src/test/regress/expected/isolation_reference_copy_vs_all.out index 1cba83a40..67dab2685 100644 --- a/src/test/regress/expected/isolation_reference_copy_vs_all.out +++ b/src/test/regress/expected/isolation_reference_copy_vs_all.out @@ -60,8 +60,7 @@ step s1-initialize: COPY reference_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-copy: COPY reference_copy FROM PROGRAM 'echo 5, f, 5 && echo 6, g, 6 && echo 7, h, 7 && echo 8, i, 8 && echo 9, j, 9' WITH CSV; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -382,8 +381,7 @@ create_reference_table step s1-initialize: COPY reference_copy FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data diff --git a/src/test/regress/expected/isolation_select_vs_all.out b/src/test/regress/expected/isolation_select_vs_all.out index 3cd3b8dae..f9575c141 100644 --- a/src/test/regress/expected/isolation_select_vs_all.out +++ b/src/test/regress/expected/isolation_select_vs_all.out @@ -61,7 +61,7 @@ id data int_data 1 b 1 step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -153,7 +153,7 @@ id data int_data 3 d 3 4 e 4 step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -179,7 +179,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -209,7 +209,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -243,7 +243,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -254,7 +254,7 @@ id data int_data id data int_d 3 d 3 3 d 3 4 e 4 4 e 4 step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1772,7 +1772,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1799,7 +1799,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1826,7 +1826,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1853,7 +1853,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1880,7 +1880,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1908,7 +1908,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1934,7 +1934,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -1967,7 +1967,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-ddl-create-index: CREATE INDEX select_append_index ON select_append(id); step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2000,7 +2000,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2032,7 +2032,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2066,7 +2066,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-ddl-add-column: ALTER TABLE select_append ADD new_column int DEFAULT 0; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data new_column id data int_data new_column @@ -2099,7 +2099,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2132,7 +2132,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2162,7 +2162,7 @@ create_distributed_table step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, 1 && echo 2, c, 2 && echo 3, d, 3 && echo 4, e, 4' WITH CSV; step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2190,7 +2190,7 @@ step s1-drop: DROP TABLE select_append; step s1-create-non-distributed-table: CREATE TABLE select_append(id integer, data text, int_data int); step s1-begin: BEGIN; step s1-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2216,7 +2216,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-insert: INSERT INTO select_append VALUES(0, 'k', 0); step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2243,7 +2243,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-insert-select: INSERT INTO select_append SELECT * FROM select_append; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2270,7 +2270,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-update: UPDATE select_append SET data = 'l' WHERE id = 0; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2297,7 +2297,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-delete: DELETE FROM select_append WHERE id = 1; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2324,7 +2324,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-truncate: TRUNCATE select_append; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; step s1-commit: COMMIT; @@ -2347,7 +2347,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-drop: DROP TABLE select_append; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; step s1-commit: COMMIT; @@ -2367,7 +2367,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-ddl-create-index: CREATE INDEX select_append_index ON select_append(id); step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2400,7 +2400,7 @@ step s1-ddl-create-index: CREATE INDEX select_append_index ON select_append(id); step s1-begin: BEGIN; step s1-ddl-drop-index: DROP INDEX select_append_index; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; step s1-commit: COMMIT; @@ -2433,7 +2433,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-ddl-add-column: ALTER TABLE select_append ADD new_column int DEFAULT 0; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; step s1-commit: COMMIT; @@ -2467,7 +2467,7 @@ step s1-ddl-add-column: ALTER TABLE select_append ADD new_column int DEFAULT 0; step s1-begin: BEGIN; step s1-ddl-drop-column: ALTER TABLE select_append DROP new_column; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; step s1-commit: COMMIT; @@ -2500,7 +2500,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-ddl-rename-column: ALTER TABLE select_append RENAME data TO new_column; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; step s1-commit: COMMIT; @@ -2536,7 +2536,7 @@ citus_total_relation_size 32768 step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2563,7 +2563,7 @@ step s1-initialize: COPY select_append FROM PROGRAM 'echo 0, a, 0 && echo 1, b, step s1-begin: BEGIN; step s1-master-modify-multiple-shards: DELETE FROM select_append; step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data @@ -2594,7 +2594,7 @@ create_distributed_table step s2-task-tracker-select: - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; id data int_data id data int_data diff --git a/src/test/regress/expected/limit_intermediate_size.out b/src/test/regress/expected/limit_intermediate_size.out index 335eae25a..1e410d5ea 100644 --- a/src/test/regress/expected/limit_intermediate_size.out +++ b/src/test/regress/expected/limit_intermediate_size.out @@ -1,5 +1,4 @@ SET citus.enable_repartition_joins to ON; -SET citus.task_executor_type to 'task-tracker'; -- prevent PG 11 - PG 12 outputs to diverge SET citus.enable_cte_inlining TO false; SET citus.max_intermediate_result_size TO 2; @@ -38,9 +37,20 @@ FROM ORDER BY 1,2 LIMIT 10; -ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 9 kB) -DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place. -HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable. + user_id | value_2 +--------------------------------------------------------------------- + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 + 1 | 0 +(10 rows) + -- router queries should be able to get limitted too SET citus.max_intermediate_result_size TO 2; -- this should pass, since we fetch small portions in each subplan @@ -105,10 +115,12 @@ WITH cte AS ( SELECT * FROM cte2, cte3 WHERE cte2.user_id = cte3.user_id AND cte2.user_id = 1 AND EXISTS (select * from cte2, cte3) ) -SELECT * FROM cte WHERE EXISTS (select * from cte); -ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 4 kB) -DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place. -HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable. +SELECT count(*) FROM cte WHERE EXISTS (select * from cte); + count +--------------------------------------------------------------------- + 105 +(1 row) + SET citus.max_intermediate_result_size TO 3; -- this should fail since the cte-subplan exceeds the limit even if the -- cte2 and cte3 does not @@ -121,10 +133,12 @@ WITH cte AS ( ) SELECT * FROM cte2, cte3 WHERE cte2.value_1 IN (SELECT value_2 FROM cte3) ) -SELECT * FROM cte; -ERROR: the intermediate result size exceeds citus.max_intermediate_result_size (currently 3 kB) -DETAIL: Citus restricts the size of intermediate results of complex subqueries and CTEs to avoid accidentally pulling large result sets into once place. -HINT: To run the current query, set citus.max_intermediate_result_size to a higher value or -1 to disable. +SELECT count(*) FROM cte; + count +--------------------------------------------------------------------- + 1824 +(1 row) + -- this will fail in real_time_executor SET citus.max_intermediate_result_size TO 2; WITH cte AS ( diff --git a/src/test/regress/expected/local_shard_execution.out b/src/test/regress/expected/local_shard_execution.out index b4cd9f860..159e74b47 100644 --- a/src/test/regress/expected/local_shard_execution.out +++ b/src/test/regress/expected/local_shard_execution.out @@ -1403,9 +1403,13 @@ BEGIN; NOTICE: executing the command locally: DELETE FROM local_shard_execution.distributed_table_1470003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) SET LOCAL citus.task_executor_type = 'task-tracker'; SELECT count(*) FROM distributed_table; -ERROR: cannot execute command because a local execution has accessed a placement in the transaction -DETAIL: Some parallel commands cannot be executed if a previous command has already been executed locally -HINT: Try re-running the transaction with "SET LOCAL citus.enable_local_execution TO OFF;" +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470003 distributed_table WHERE true + count +--------------------------------------------------------------------- + 100 +(1 row) + ROLLBACK; -- local execution should not be executed locally -- becase a task-tracker query has already been executed @@ -1413,6 +1417,8 @@ BEGIN; SET LOCAL citus.task_executor_type = 'task-tracker'; SET LOCAL client_min_messages TO INFO; SELECT count(*) FROM distributed_table; +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470001 distributed_table WHERE true +NOTICE: executing the command locally: SELECT count(*) AS count FROM local_shard_execution.distributed_table_1470003 distributed_table WHERE true count --------------------------------------------------------------------- 101 @@ -1420,6 +1426,7 @@ BEGIN; SET LOCAL client_min_messages TO LOG; DELETE FROM distributed_table WHERE key = 500; +NOTICE: executing the command locally: DELETE FROM local_shard_execution.distributed_table_1470003 distributed_table WHERE (key OPERATOR(pg_catalog.=) 500) ROLLBACK; -- probably not a realistic case since views are not very -- well supported with MX diff --git a/src/test/regress/expected/multi_binary_master_copy_format.out b/src/test/regress/expected/multi_binary_master_copy_format.out index 3d506408e..baee865a0 100644 --- a/src/test/regress/expected/multi_binary_master_copy_format.out +++ b/src/test/regress/expected/multi_binary_master_copy_format.out @@ -4,7 +4,6 @@ SET citus.next_shard_id TO 430000; -- Try binary master copy for different executors SET citus.binary_master_copy_format TO 'on'; -SET citus.task_executor_type TO 'task-tracker'; SELECT count(*) FROM lineitem; count --------------------------------------------------------------------- @@ -18,7 +17,6 @@ SELECT l_shipmode FROM lineitem WHERE l_partkey = 67310 OR l_partkey = 155190; MAIL (2 rows) -RESET citus.task_executor_type; SELECT count(*) FROM lineitem; count --------------------------------------------------------------------- diff --git a/src/test/regress/expected/multi_complex_expressions.out b/src/test/regress/expected/multi_complex_expressions.out index b77a637c6..5740041f2 100644 --- a/src/test/regress/expected/multi_complex_expressions.out +++ b/src/test/regress/expected/multi_complex_expressions.out @@ -384,7 +384,6 @@ SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20; (10 rows) -- LIMIT/OFFSET with a subquery -SET citus.task_executor_type TO 'task-tracker'; SELECT customer_keys.o_custkey, SUM(order_count) AS total_order_count @@ -460,7 +459,6 @@ LIMIT 10 OFFSET 20; 1453 | 1504 (10 rows) -RESET citus.task_executor_type; SET client_min_messages TO DEBUG1; -- Ensure that we push down LIMIT and OFFSET properly -- No Group-By -> Push Down diff --git a/src/test/regress/expected/multi_complex_expressions_0.out b/src/test/regress/expected/multi_complex_expressions_0.out index b020b0055..b9a1b42e8 100644 --- a/src/test/regress/expected/multi_complex_expressions_0.out +++ b/src/test/regress/expected/multi_complex_expressions_0.out @@ -370,7 +370,6 @@ SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20; (10 rows) -- LIMIT/OFFSET with a subquery -SET citus.task_executor_type TO 'task-tracker'; SELECT customer_keys.o_custkey, SUM(order_count) AS total_order_count @@ -446,7 +445,6 @@ LIMIT 10 OFFSET 20; 1453 | 1504 (10 rows) -RESET citus.task_executor_type; SET client_min_messages TO DEBUG1; -- Ensure that we push down LIMIT and OFFSET properly -- No Group-By -> Push Down diff --git a/src/test/regress/expected/multi_cross_shard.out b/src/test/regress/expected/multi_cross_shard.out index 7ee7fb374..35becc66c 100644 --- a/src/test/regress/expected/multi_cross_shard.out +++ b/src/test/regress/expected/multi_cross_shard.out @@ -3,6 +3,7 @@ -- -- Tests to log cross shard queries according to error log level -- +SET citus.enable_repartition_joins to ON; -- Create a distributed table and add data to it CREATE TABLE multi_task_table ( @@ -173,8 +174,7 @@ INSERT INTO tt1 VALUES(2, 'Mehmet'); INSERT INTO tt2 VALUES(1, 'Ahmet', 5); INSERT INTO tt2 VALUES(2, 'Mehmet', 15); -- Should notice since it is a task-tracker query -SET citus.task_executor_type to "task-tracker"; -SELECT tt1.id, tt2.count from tt1,tt2 where tt1.id = tt2.id; +SELECT tt1.id, tt2.count from tt1,tt2 where tt1.id = tt2.id ORDER BY 1; NOTICE: multi-task query about to be executed HINT: Queries are split to multiple tasks if they have to be split into several queries on the workers. id | count diff --git a/src/test/regress/expected/multi_explain.out b/src/test/regress/expected/multi_explain.out index 1612446c7..084a2867a 100644 --- a/src/test/regress/expected/multi_explain.out +++ b/src/test/regress/expected/multi_explain.out @@ -3,8 +3,8 @@ -- SET citus.next_shard_id TO 570000; \a\t -RESET citus.task_executor_type; SET citus.explain_distributed_queries TO on; +SET citus.enable_repartition_joins to ON; -- Function that parses explain output as JSON CREATE FUNCTION explain_json(query text) RETURNS jsonb @@ -1047,11 +1047,10 @@ Custom Scan (Citus Adaptive) -> Hash -> Seq Scan on orders_hash_part_360045 orders_hash_part -- Test track tracker -SET citus.task_executor_type TO 'task-tracker'; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 1 Tasks Shown: All -> Task @@ -1067,7 +1066,7 @@ EXPLAIN (COSTS FALSE) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 1 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -1093,7 +1092,7 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "Citus Task-Tracker", + "Custom Plan Provider": "Citus Adaptive", "Parallel Aware": false, "Distributed Query": { "Job": { @@ -1142,7 +1141,7 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - Citus Task-Tracker + Citus Adaptive false @@ -1201,7 +1200,7 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "Citus Task-Tracker" + Custom Plan Provider: "Citus Adaptive" Parallel Aware: false Distributed Query: Job: @@ -1218,7 +1217,7 @@ Aggregate -- ensure distributed plans don't break EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem; Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 2 Tasks Shown: One of 2 -> Task @@ -1230,7 +1229,7 @@ PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 1 Tasks Shown: All -> Task @@ -1238,7 +1237,6 @@ Aggregate -> Aggregate -> Seq Scan on lineitem_290001 lineitem Filter: (l_orderkey > 9030) -RESET citus.task_executor_type; PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; EXPLAIN EXECUTE router_executor_query; Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=100000 width=18) diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index 8fcfbb9e7..2aefb1a58 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -418,9 +418,15 @@ SELECT * FROM print_extension_changes(); -- Snapshot of state at 9.5-1 ALTER EXTENSION citus UPDATE TO '9.5-1'; SELECT * FROM print_extension_changes(); - previous_object | current_object + previous_object | current_object --------------------------------------------------------------------- -(0 rows) + function task_tracker_assign_task(bigint,integer,text) | + function task_tracker_cleanup_job(bigint) | + function task_tracker_conninfo_cache_invalidate() | + function task_tracker_task_status(bigint,integer) | + function worker_execute_sql_task(bigint,integer,text,boolean) | + function worker_merge_files_and_run_query(bigint,integer,text,text) | +(6 rows) DROP TABLE prev_objects, extension_diff; -- show running version diff --git a/src/test/regress/expected/multi_follower_task_tracker.out b/src/test/regress/expected/multi_follower_task_tracker.out deleted file mode 100644 index f6b75bd40..000000000 --- a/src/test/regress/expected/multi_follower_task_tracker.out +++ /dev/null @@ -1,27 +0,0 @@ -\c - - - :master_port --- do some setup -CREATE TABLE tab(a int, b int); -SELECT create_distributed_table('tab', 'a'); - create_distributed_table ---------------------------------------------------------------------- - -(1 row) - -INSERT INTO tab (a, b) VALUES (1, 1); -INSERT INTO tab (a, b) VALUES (1, 2); -\c - - - :follower_master_port -RESET citus.task_executor_type; -SELECT * FROM tab; - a | b ---------------------------------------------------------------------- - 1 | 1 - 1 | 2 -(2 rows) - -SET citus.task_executor_type TO 'task-tracker'; -SELECT * FROM tab; -ERROR: task tracker queries are not allowed while citus.use_secondary_nodes is 'always' -HINT: try setting citus.task_executor_type TO 'adaptive' --- clean up -\c - - - :master_port -DROP TABLE tab; diff --git a/src/test/regress/expected/multi_hash_pruning.out b/src/test/regress/expected/multi_hash_pruning.out index e1c8a51b4..50a9a25ad 100644 --- a/src/test/regress/expected/multi_hash_pruning.out +++ b/src/test/regress/expected/multi_hash_pruning.out @@ -1080,7 +1080,6 @@ DEBUG: assigned task to node localhost:xxxxx 4 (1 row) -SET citus.task_executor_type TO 'adaptive'; -- Check that subquery NOT is pruned when ANDed to a valid constraint SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey IN (1,2) AND o_custkey NOT IN (SELECT o_custkey FROM orders_hash_partitioned WHERE o_orderkey = 1); diff --git a/src/test/regress/expected/multi_join_order_additional.out b/src/test/regress/expected/multi_join_order_additional.out index feb7d7576..6dae378df 100644 --- a/src/test/regress/expected/multi_join_order_additional.out +++ b/src/test/regress/expected/multi_join_order_additional.out @@ -6,6 +6,7 @@ SET citus.next_shard_id TO 650000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise +SET citus.enable_repartition_joins to ON; SET citus.shard_count to 2; SET citus.shard_replication_factor to 1; RESET client_min_messages; @@ -79,7 +80,7 @@ DEBUG: join prunable for intervals [1,5986] and [8997,14947] DEBUG: join prunable for intervals [8997,14947] and [1,5986] QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (2 rows) @@ -95,7 +96,7 @@ LOG: join order: [ "lineitem" ][ local partition join "orders" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -110,7 +111,7 @@ LOG: join order: [ "orders" ][ single range partition join "lineitem_hash" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -122,7 +123,7 @@ LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -134,7 +135,7 @@ LOG: join order: [ "customer_hash" ][ reference join "nation" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -147,7 +148,7 @@ LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -160,7 +161,7 @@ LOG: join order: [ "orders" ][ dual partition join "customer_hash" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -173,7 +174,7 @@ LOG: join order: [ "orders_hash" ][ single range partition join "customer_appen QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -198,7 +199,7 @@ LOG: join order: [ "users_table" ][ local partition join "events_table" ][ loca QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) diff --git a/src/test/regress/expected/multi_join_order_tpch_repartition.out b/src/test/regress/expected/multi_join_order_tpch_repartition.out index a0b7a72d4..5d7b5c2c1 100644 --- a/src/test/regress/expected/multi_join_order_tpch_repartition.out +++ b/src/test/regress/expected/multi_join_order_tpch_repartition.out @@ -6,6 +6,7 @@ SET citus.next_shard_id TO 660000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise +SET citus.enable_repartition_joins to ON; SET client_min_messages TO LOG; -- The following queries are basically the same as the ones in tpch_small -- except that more data has been loaded into customer and part tables. Therefore, @@ -25,7 +26,7 @@ LOG: join order: [ "lineitem" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -60,7 +61,7 @@ LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single range Sort Key: (sum(remote_scan.revenue)) DESC, remote_scan.o_orderdate -> HashAggregate Group Key: remote_scan.l_orderkey, remote_scan.o_orderdate, remote_scan.o_shippriority - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (6 rows) @@ -104,7 +105,7 @@ LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single range Sort Key: (sum(remote_scan.revenue)) DESC -> HashAggregate Group Key: remote_scan.c_custkey, remote_scan.c_name, remote_scan.c_acctbal, remote_scan.c_phone, remote_scan.n_name, remote_scan.c_address, remote_scan.c_comment - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (6 rows) @@ -143,7 +144,7 @@ LOG: join order: [ "lineitem" ][ single range partition join "part_append" ] QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (3 rows) @@ -164,7 +165,7 @@ LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single range --------------------------------------------------------------------- HashAggregate Group Key: remote_scan.l_partkey - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) explain statements for distributed queries are not enabled (4 rows) diff --git a/src/test/regress/expected/multi_multiuser.out b/src/test/regress/expected/multi_multiuser.out index 0b84a601e..0d7630fe6 100644 --- a/src/test/regress/expected/multi_multiuser.out +++ b/src/test/regress/expected/multi_multiuser.out @@ -141,7 +141,7 @@ SELECT count(*) FROM test WHERE id = 1; 1 (1 row) -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; SELECT count(*), min(current_user) FROM test; count | min --------------------------------------------------------------------- @@ -155,7 +155,6 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b. 0 (1 row) -SET citus.task_executor_type TO 'adaptive'; SET citus.enable_repartition_joins TO true; SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; count @@ -167,18 +166,10 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b. COPY "postgresql.conf" TO STDOUT WITH (format transmit); ERROR: operation is not allowed HINT: Run the command with a superuser. -RESET citus.task_executor_type; -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); ERROR: operation is not allowed HINT: Run the command with a superuser. --- create a task that other users should not be able to inspect -SELECT task_tracker_assign_task(1, 1, 'SELECT 1'); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -- check read permission SET ROLE read_access; -- should be allowed to run commands, as the current user @@ -229,7 +220,7 @@ SELECT count(*) FROM test WHERE id = 1; 1 (1 row) -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; SELECT count(*), min(current_user) FROM test; count | min --------------------------------------------------------------------- @@ -243,7 +234,6 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b. 0 (1 row) -SET citus.task_executor_type TO 'adaptive'; SET citus.enable_repartition_joins TO true; SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; count @@ -255,14 +245,6 @@ SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b. COPY "postgresql.conf" TO STDOUT WITH (format transmit); ERROR: operation is not allowed HINT: Run the command with a superuser. --- should not be able to access tasks or jobs belonging to a different user -SELECT task_tracker_task_status(1, 1); -ERROR: could not find the worker task -DETAIL: Task jobId: 1 and taskId: 1 -SELECT task_tracker_assign_task(1, 2, 'SELECT 1'); -ERROR: must be owner of schema pg_merge_job_0001 -SELECT task_tracker_cleanup_job(1); -ERROR: must be owner of schema pg_merge_job_0001 -- should not be allowed to take aggressive locks on table BEGIN; SELECT lock_relation_if_exists('test', 'ACCESS SHARE'); @@ -274,7 +256,6 @@ SELECT lock_relation_if_exists('test', 'ACCESS SHARE'); SELECT lock_relation_if_exists('test', 'EXCLUSIVE'); ERROR: permission denied for table test ABORT; -RESET citus.task_executor_type; -- check no permission SET ROLE no_access; EXECUTE prepare_insert(1); @@ -287,13 +268,12 @@ SELECT count(*) FROM test; ERROR: permission denied for table test SELECT count(*) FROM test WHERE id = 1; ERROR: permission denied for table test -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; SELECT count(*), min(current_user) FROM test; ERROR: permission denied for table test -- test re-partition query SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; ERROR: permission denied for table test -SET citus.task_executor_type TO 'adaptive'; SET citus.enable_repartition_joins TO true; SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; ERROR: permission denied for table test @@ -301,7 +281,6 @@ ERROR: permission denied for table test COPY "postgresql.conf" TO STDOUT WITH (format transmit); ERROR: operation is not allowed HINT: Run the command with a superuser. -RESET citus.task_executor_type; -- should be able to use intermediate results as any user BEGIN; SELECT create_intermediate_result('topten', 'SELECT s FROM generate_series(1,10) s'); @@ -365,12 +344,6 @@ SELECT result FROM run_command_on_workers($$SELECT tableowner FROM pg_tables WHE full_access (2 rows) -SELECT task_tracker_cleanup_job(1); - task_tracker_cleanup_job ---------------------------------------------------------------------- - -(1 row) - -- table should be distributable by super user when it has data in there SET ROLE full_access; CREATE TABLE my_table_with_data (id integer, val integer); @@ -681,6 +654,10 @@ ERROR: permission denied for function worker_cleanup_job_schema_cache RESET ROLE; -- to test access to files created during repartition we will create some on worker 1 \c - - - :worker_1_port +CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) + RETURNS void + AS 'citus' + LANGUAGE C STRICT; SET ROLE full_access; SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]); worker_hash_partition_table @@ -691,6 +668,10 @@ SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS RESET ROLE; -- all attempts for transfer are initiated from other workers \c - - - :worker_2_port +CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) + RETURNS void + AS 'citus' + LANGUAGE C STRICT; -- super user should not be able to copy files created by a user SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port); WARNING: could not open file "base/pgsql_job_cache/job_0042/task_000001/p_00001.xxxx": No such file or directory @@ -718,22 +699,12 @@ SET ROLE usage_access; SELECT worker_merge_files_into_table(42, 1, ARRAY['a'], ARRAY['integer']); ERROR: job schema does not exist DETAIL: must be superuser to use public schema -RESET ROLE; -SET ROLE full_access; --- use the side effect of this function to have a schema to use, otherwise only the super --- user could call worker_merge_files_into_table and store the results in public, which is --- not what we want -SELECT task_tracker_assign_task(42, 1, 'SELECT 1'); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - RESET ROLE; -- test that no other user can merge the downloaded file after the task is being tracked SET ROLE usage_access; SELECT worker_merge_files_into_table(42, 1, ARRAY['a'], ARRAY['integer']); -ERROR: must be owner of schema pg_merge_job_0042 +ERROR: job schema does not exist +DETAIL: must be superuser to use public schema RESET ROLE; -- test that the super user is unable to read the contents of the intermediate file, -- although it does create the table @@ -745,106 +716,45 @@ WARNING: Task file "task_000001.xxxx" does not have expected suffix ".10" (1 row) SELECT count(*) FROM pg_merge_job_0042.task_000001; - count ---------------------------------------------------------------------- - 0 -(1 row) - +ERROR: relation "pg_merge_job_0042.task_000001" does not exist DROP TABLE pg_merge_job_0042.task_000001; -- drop table so we can reuse the same files for more tests +ERROR: schema "pg_merge_job_0042" does not exist SET ROLE full_access; SELECT worker_merge_files_into_table(42, 1, ARRAY['a'], ARRAY['integer']); - worker_merge_files_into_table ---------------------------------------------------------------------- - -(1 row) - +ERROR: job schema does not exist +DETAIL: must be superuser to use public schema SELECT count(*) FROM pg_merge_job_0042.task_000001; - count ---------------------------------------------------------------------- - 25 -(1 row) - +ERROR: relation "pg_merge_job_0042.task_000001" does not exist DROP TABLE pg_merge_job_0042.task_000001; -- drop table so we can reuse the same files for more tests +ERROR: schema "pg_merge_job_0042" does not exist RESET ROLE; --- test that no other user can merge files and run query on the already fetched files -SET ROLE usage_access; -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); -ERROR: must be owner of schema pg_merge_job_0042 -RESET ROLE; --- test that the super user is unable to read the contents of the partitioned files after --- trying to merge with run query -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); -WARNING: Task file "task_000001.xxxx" does not have expected suffix ".10" - worker_merge_files_and_run_query ---------------------------------------------------------------------- - -(1 row) - SELECT count(*) FROM pg_merge_job_0042.task_000001_merge; - count ---------------------------------------------------------------------- - 0 -(1 row) - +ERROR: relation "pg_merge_job_0042.task_000001_merge" does not exist SELECT count(*) FROM pg_merge_job_0042.task_000001; - count ---------------------------------------------------------------------- - 1 -(1 row) - +ERROR: relation "pg_merge_job_0042.task_000001" does not exist DROP TABLE pg_merge_job_0042.task_000001, pg_merge_job_0042.task_000001_merge; -- drop table so we can reuse the same files for more tests --- test that the owner of the task can merge files and run query correctly -SET ROLE full_access; -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); - worker_merge_files_and_run_query ---------------------------------------------------------------------- - -(1 row) - --- test that owner of task cannot execute arbitrary sql -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000002_merge(merge_column_0 int)', - 'DROP USER usage_access' -); -ERROR: permission denied to drop role -CONTEXT: SQL statement "DROP USER usage_access" -SELECT worker_merge_files_and_run_query(42, 1, - 'DROP USER usage_access', - 'CREATE TABLE task_000002 (a) AS SELECT sum(merge_column_0) FROM task_000002_merge' -); -ERROR: permission denied to drop role -CONTEXT: SQL statement "DROP USER usage_access" +ERROR: schema "pg_merge_job_0042" does not exist SELECT count(*) FROM pg_merge_job_0042.task_000001_merge; - count ---------------------------------------------------------------------- - 25 -(1 row) - +ERROR: relation "pg_merge_job_0042.task_000001_merge" does not exist SELECT count(*) FROM pg_merge_job_0042.task_000001; - count +ERROR: relation "pg_merge_job_0042.task_000001" does not exist +DROP TABLE pg_merge_job_0042.task_000001, pg_merge_job_0042.task_000001_merge; -- drop table so we can reuse the same files for more tests +ERROR: schema "pg_merge_job_0042" does not exist +RESET ROLE; +SELECT citus_rm_job_directory(42::bigint); + citus_rm_job_directory --------------------------------------------------------------------- - 1 + +(1 row) + +\c - - - :worker_1_port +SELECT citus_rm_job_directory(42::bigint); + citus_rm_job_directory +--------------------------------------------------------------------- + (1 row) -DROP TABLE pg_merge_job_0042.task_000001, pg_merge_job_0042.task_000001_merge; -- drop table so we can reuse the same files for more tests -RESET ROLE; \c - - - :master_port -SELECT run_command_on_workers($$SELECT task_tracker_cleanup_job(42);$$); - run_command_on_workers ---------------------------------------------------------------------- - (localhost,57637,t,"") - (localhost,57638,t,"") -(2 rows) - DROP SCHEMA full_access_user_schema CASCADE; NOTICE: drop cascades to 4 other objects DETAIL: drop cascades to table full_access_user_schema.t1 diff --git a/src/test/regress/expected/multi_mx_explain.out b/src/test/regress/expected/multi_mx_explain.out index 7f043f00e..c5495a839 100644 --- a/src/test/regress/expected/multi_mx_explain.out +++ b/src/test/regress/expected/multi_mx_explain.out @@ -6,7 +6,6 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1320000; \c - - - :worker_2_port \c - - - :master_port \a\t -RESET citus.task_executor_type; SET citus.explain_distributed_queries TO on; VACUUM ANALYZE lineitem_mx; VACUUM ANALYZE orders_mx; @@ -483,12 +482,11 @@ SELECT true AS valid FROM explain_json($$ SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); t -- Test track tracker -SET citus.task_executor_type TO 'task-tracker'; SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030; Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -504,7 +502,7 @@ EXPLAIN (COSTS FALSE) AND o_custkey = c_custkey AND l_suppkey = s_suppkey; Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 16 Tasks Shown: One of 16 -> Task @@ -540,7 +538,7 @@ EXPLAIN (COSTS FALSE, FORMAT JSON) { "Node Type": "Custom Scan", "Parent Relationship": "Outer", - "Custom Plan Provider": "Citus Task-Tracker", + "Custom Plan Provider": "Citus Adaptive", "Parallel Aware": false, "Distributed Query": { "Job": { @@ -676,7 +674,7 @@ EXPLAIN (COSTS FALSE, FORMAT XML) Custom Scan Outer - Citus Task-Tracker + Citus Adaptive false @@ -808,7 +806,7 @@ EXPLAIN (COSTS FALSE, FORMAT YAML) Plans: - Node Type: "Custom Scan" Parent Relationship: "Outer" - Custom Plan Provider: "Citus Task-Tracker" + Custom Plan Provider: "Citus Adaptive" Parallel Aware: false Distributed Query: Job: diff --git a/src/test/regress/expected/multi_mx_reference_table.out b/src/test/regress/expected/multi_mx_reference_table.out index e540182f3..d76dd92bd 100644 --- a/src/test/regress/expected/multi_mx_reference_table.out +++ b/src/test/regress/expected/multi_mx_reference_table.out @@ -878,7 +878,7 @@ LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_te 2 (2 rows) -SET citus.task_executor_type to "task-tracker"; +SET citus.enable_repartition_joins to ON; SELECT colocated_table_test.value_2 FROM diff --git a/src/test/regress/expected/multi_mx_repartition_join_w1.out b/src/test/regress/expected/multi_mx_repartition_join_w1.out index c3072c5f3..f8a7aa9e0 100644 --- a/src/test/regress/expected/multi_mx_repartition_join_w1.out +++ b/src/test/regress/expected/multi_mx_repartition_join_w1.out @@ -3,7 +3,8 @@ -- concurrent test runs the same query on :worker_2_port. Note that, both -- tests use the same sequence ids but the queries should not fail. \c - - - :worker_1_port -SET citus.task_executor_type TO "task-tracker"; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; CREATE TEMP TABLE t1 AS SELECT l1.l_comment diff --git a/src/test/regress/expected/multi_mx_repartition_join_w2.out b/src/test/regress/expected/multi_mx_repartition_join_w2.out index a643cd90d..8a5b65af7 100644 --- a/src/test/regress/expected/multi_mx_repartition_join_w2.out +++ b/src/test/regress/expected/multi_mx_repartition_join_w2.out @@ -3,7 +3,8 @@ -- concurrent test runs the same query on :worker_1_port. Note that, both -- tests use the same sequence ids but the queries should not fail. \c - - - :worker_2_port -SET citus.task_executor_type TO "task-tracker"; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; CREATE TEMP TABLE t1 AS SELECT l1.l_comment diff --git a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out index 19776411c..9502e8fcc 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_prepare.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_prepare.out @@ -153,6 +153,7 @@ INSERT INTO repartition_udt_other values (11, '(2,2)'::test_udt, 'foo'); INSERT INTO repartition_udt_other values (12, '(2,3)'::test_udt, 'foo'); SET client_min_messages = LOG; SET citus.task_executor_type = 'task-tracker'; +SET citus.enable_repartition_joins to ON; -- Query that should result in a repartition -- join on int column, and be empty. SELECT * FROM repartition_udt JOIN repartition_udt_other @@ -170,7 +171,7 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ] QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob diff --git a/src/test/regress/expected/multi_mx_repartition_udt_w1.out b/src/test/regress/expected/multi_mx_repartition_udt_w1.out index 2095866e8..0812e72bf 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_w1.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_w1.out @@ -4,7 +4,8 @@ \c - - - :worker_1_port SET client_min_messages = LOG; -- Query that should result in a repartition join on UDT column. -SET citus.task_executor_type = 'task-tracker'; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; SET citus.log_multi_join_order = true; -- Query that should result in a repartition -- join on int column, and be empty diff --git a/src/test/regress/expected/multi_mx_repartition_udt_w2.out b/src/test/regress/expected/multi_mx_repartition_udt_w2.out index 547c62c5b..dd70b46d1 100644 --- a/src/test/regress/expected/multi_mx_repartition_udt_w2.out +++ b/src/test/regress/expected/multi_mx_repartition_udt_w2.out @@ -4,7 +4,8 @@ \c - - - :worker_2_port SET client_min_messages = LOG; -- Query that should result in a repartition join on UDT column. -SET citus.task_executor_type = 'task-tracker'; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; SET citus.log_multi_join_order = true; -- Query that should result in a repartition -- join on int column, and be empty. diff --git a/src/test/regress/expected/multi_mx_router_planner.out b/src/test/regress/expected/multi_mx_router_planner.out index 24c22d4c6..30a51b752 100644 --- a/src/test/regress/expected/multi_mx_router_planner.out +++ b/src/test/regress/expected/multi_mx_router_planner.out @@ -59,7 +59,6 @@ INSERT INTO articles_hash_mx VALUES (47, 7, 'abeyance', 1772); INSERT INTO articles_hash_mx VALUES (48, 8, 'alkylic', 18610); INSERT INTO articles_hash_mx VALUES (49, 9, 'anyone', 2681); INSERT INTO articles_hash_mx VALUES (50, 10, 'anjanette', 19519); -RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- insert a single row for the test INSERT INTO articles_single_shard_hash_mx VALUES (50, 10, 'anjanette', 19519); @@ -876,7 +875,6 @@ ORDER BY 1,2,3,4; 43 | 3 | affixal | 12723 (15 rows) -RESET citus.task_executor_type; -- Test various filtering options for router plannable check SET client_min_messages to 'DEBUG2'; -- this is definitely single shard @@ -1496,8 +1494,7 @@ DEBUG: Router planner cannot handle multi-shard select queries -- router planner/executor is disabled for task-tracker executor -- following query is router plannable, but router planner is disabled -- TODO: Uncomment once we fix task-tracker issue ---SET citus.task_executor_type to 'task-tracker'; ---SELECT id +----SELECT id -- FROM articles_hash_mx -- WHERE author_id = 1; -- insert query is router plannable even under task-tracker diff --git a/src/test/regress/expected/multi_mx_schema_support.out b/src/test/regress/expected/multi_mx_schema_support.out index f7669fb70..fca95340c 100644 --- a/src/test/regress/expected/multi_mx_schema_support.out +++ b/src/test/regress/expected/multi_mx_schema_support.out @@ -160,7 +160,7 @@ SELECT * FROM nation_hash WHERE n_nationkey OPERATOR(===) 1; (1 row) SELECT * FROM citus_mx_test_schema.nation_hash_collation_search_path ORDER BY 1; - n_nationkey | n_name | n_regionkey | n_comment + n_nationkey | n_name | n_regionkey | n_comment --------------------------------------------------------------------- 0 | ALGERIA | 0 | haggle. carefully final deposits detect slyly agai 1 | ARGENTINA | 1 | al foxes promise slyly according to the regular accounts. bold requests alon @@ -171,7 +171,7 @@ SELECT * FROM citus_mx_test_schema.nation_hash_collation_search_path ORDER BY 1; (6 rows) SELECT n_comment FROM citus_mx_test_schema.nation_hash_collation_search_path ORDER BY n_comment COLLATE citus_mx_test_schema.english; - n_comment + n_comment --------------------------------------------------------------------- al foxes promise slyly according to the regular accounts. bold requests alon eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold @@ -183,7 +183,7 @@ SELECT n_comment FROM citus_mx_test_schema.nation_hash_collation_search_path ORD SET search_path TO citus_mx_test_schema; SELECT * FROM nation_hash_collation_search_path ORDER BY 1 DESC; - n_nationkey | n_name | n_regionkey | n_comment + n_nationkey | n_name | n_regionkey | n_comment --------------------------------------------------------------------- 5 | ETHIOPIA | 0 | ven packages wake quickly. regu 4 | EGYPT | 4 | y above the carefully unusual theodolites. final dugouts are quickly across the furiously regular d @@ -194,7 +194,7 @@ SELECT * FROM nation_hash_collation_search_path ORDER BY 1 DESC; (6 rows) SELECT n_comment FROM nation_hash_collation_search_path ORDER BY n_comment COLLATE english; - n_comment + n_comment --------------------------------------------------------------------- al foxes promise slyly according to the regular accounts. bold requests alon eas hang ironic, silent packages. slyly regular packages are furiously over the tithes. fluffily bold @@ -218,6 +218,7 @@ SELECT * FROM nation_hash_composite_types WHERE test_col = '(a,a)'::new_composit 0 | ALGERIA | 0 | haggle. carefully final deposits detect slyly agai | (a,a) (1 row) +SET citus.enable_repartition_joins to ON; -- check when search_path is public, -- join of two tables which are in different schemas, -- join on partition column @@ -279,7 +280,6 @@ WHERE (1 row) -- single repartition joins -SET citus.task_executor_type TO "task-tracker"; -- check when search_path is public, -- join of two tables which are in different schemas, -- join on partition column and non-partition column @@ -372,7 +372,6 @@ WHERE (1 row) -- set task_executor back to adaptive -SET citus.task_executor_type TO "adaptive"; -- connect to the master and do some test -- regarding DDL support on schemas where -- the search_path is set diff --git a/src/test/regress/expected/multi_null_minmax_value_pruning.out b/src/test/regress/expected/multi_null_minmax_value_pruning.out index ea5b81844..ba56e8a50 100644 --- a/src/test/regress/expected/multi_null_minmax_value_pruning.out +++ b/src/test/regress/expected/multi_null_minmax_value_pruning.out @@ -7,7 +7,6 @@ SET client_min_messages TO DEBUG2; SET citus.explain_all_tasks TO on; -- to avoid differing explain output - executor doesn't matter, -- because were testing pruning here. -RESET citus.task_executor_type; -- Change configuration to treat lineitem and orders tables as large SET citus.log_multi_join_order to true; SET citus.enable_repartition_joins to ON; diff --git a/src/test/regress/expected/multi_partitioning.out b/src/test/regress/expected/multi_partitioning.out index 18691286c..403404321 100644 --- a/src/test/regress/expected/multi_partitioning.out +++ b/src/test/regress/expected/multi_partitioning.out @@ -4,6 +4,7 @@ SET citus.next_shard_id TO 1660000; SET citus.shard_count TO 4; SET citus.shard_replication_factor TO 1; +SET citus.enable_repartition_joins to ON; -- -- Distributed Partitioned Table Creation Tests -- @@ -1299,7 +1300,6 @@ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass COMMIT; -- test locks on task-tracker SELECT -SET citus.task_executor_type TO 'task-tracker'; BEGIN; SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; id | ref_id | time | id | ref_id | time @@ -1315,7 +1315,6 @@ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass (3 rows) COMMIT; -RESET citus.task_executor_type; -- test locks on INSERT BEGIN; INSERT INTO partitioning_locks VALUES(1, 1, '2009-01-01'); diff --git a/src/test/regress/expected/multi_prepare_plsql.out b/src/test/regress/expected/multi_prepare_plsql.out index b8e8608dc..36073f55d 100644 --- a/src/test/regress/expected/multi_prepare_plsql.out +++ b/src/test/regress/expected/multi_prepare_plsql.out @@ -137,7 +137,6 @@ BEGIN l_year; END; $$ LANGUAGE plpgsql; -SET citus.task_executor_type TO 'task-tracker'; SET client_min_messages TO INFO; -- now, run PL/pgsql functions SELECT plpgsql_test_1(); @@ -252,7 +251,6 @@ SELECT plpgsql_test_2(); -- run the tests which do not require re-partition -- with real-time executor -RESET citus.task_executor_type; -- now, run PL/pgsql functions SELECT plpgsql_test_1(); plpgsql_test_1 @@ -742,7 +740,6 @@ SELECT real_time_partition_column_select(6); (4 rows) -- check task-tracker executor -SET citus.task_executor_type TO 'task-tracker'; CREATE FUNCTION task_tracker_non_partition_column_select(value_arg int) RETURNS TABLE(key int, value int) AS $$ DECLARE @@ -875,7 +872,6 @@ SELECT task_tracker_partition_column_select(6); (6,) (4 rows) -RESET citus.task_executor_type; -- check updates CREATE FUNCTION partition_parameter_update(int, int) RETURNS void as $$ BEGIN diff --git a/src/test/regress/expected/multi_prepare_sql.out b/src/test/regress/expected/multi_prepare_sql.out index 02fd99ae1..82bf30210 100644 --- a/src/test/regress/expected/multi_prepare_sql.out +++ b/src/test/regress/expected/multi_prepare_sql.out @@ -99,7 +99,6 @@ ORDER BY supp_nation, cust_nation, l_year; -SET citus.task_executor_type TO 'task-tracker'; SET client_min_messages TO INFO; -- execute prepared statements EXECUTE prepared_test_1; @@ -221,7 +220,6 @@ SELECT * from prepared_sql_test_7; (1 row) -- now, run some of the tests with real-time executor -RESET citus.task_executor_type; -- execute prepared statements EXECUTE prepared_test_1; count @@ -766,7 +764,6 @@ EXECUTE prepared_real_time_partition_column_select(6); (4 rows) -- check task-tracker executor -SET citus.task_executor_type TO 'task-tracker'; PREPARE prepared_task_tracker_non_partition_column_select(int) AS SELECT prepare_table.key, @@ -885,7 +882,6 @@ EXECUTE prepared_task_tracker_partition_column_select(6); 6 | (4 rows) -RESET citus.task_executor_type; -- check updates PREPARE prepared_partition_parameter_update(int, int) AS UPDATE prepare_table SET value = $2 WHERE key = $1; diff --git a/src/test/regress/expected/multi_reference_table.out b/src/test/regress/expected/multi_reference_table.out index 57dd97344..12d7c35b9 100644 --- a/src/test/regress/expected/multi_reference_table.out +++ b/src/test/regress/expected/multi_reference_table.out @@ -1086,13 +1086,14 @@ LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_te 2 (2 rows) -SET citus.task_executor_type to "task-tracker"; +SET citus.enable_repartition_joins to ON; SELECT colocated_table_test.value_2 FROM reference_table_test, colocated_table_test, colocated_table_test_2 WHERE - colocated_table_test.value_2 = colocated_table_test_2.value_2 AND colocated_table_test.value_2 = reference_table_test.value_2; + colocated_table_test.value_2 = colocated_table_test_2.value_2 AND colocated_table_test.value_2 = reference_table_test.value_2 +ORDER BY colocated_table_test.value_2; LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ dual partition join "colocated_table_test_2" ] value_2 --------------------------------------------------------------------- @@ -1105,7 +1106,8 @@ SELECT FROM reference_table_test, colocated_table_test, colocated_table_test_2 WHERE - colocated_table_test.value_1 = reference_table_test.value_1 AND colocated_table_test_2.value_1 = reference_table_test.value_1; + colocated_table_test.value_1 = reference_table_test.value_1 AND colocated_table_test_2.value_1 = reference_table_test.value_1 +ORDER BY reference_table_test.value_2; LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ dual partition join "colocated_table_test_2" ] value_2 --------------------------------------------------------------------- @@ -1115,7 +1117,6 @@ LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_te SET citus.log_multi_join_order TO FALSE; SET citus.shard_count TO DEFAULT; -SET citus.task_executor_type to "adaptive"; -- some INSERT .. SELECT queries that involve both hash distributed and reference tables -- should go via coordinator since we're inserting into reference table where -- not all the participants are reference tables diff --git a/src/test/regress/expected/multi_repartition_join_planning.out b/src/test/regress/expected/multi_repartition_join_planning.out index 120be21ff..c20cbc2b8 100644 --- a/src/test/regress/expected/multi_repartition_join_planning.out +++ b/src/test/regress/expected/multi_repartition_join_planning.out @@ -7,6 +7,7 @@ -- executor here, as we cannot run repartition jobs with real time executor. SET citus.next_shard_id TO 690000; SET citus.enable_unique_job_ids TO off; +SET citus.enable_repartition_joins to ON; create schema repartition_join; DROP TABLE IF EXISTS repartition_join.order_line; NOTICE: table "order_line" does not exist, skipping @@ -40,8 +41,7 @@ SELECT create_distributed_table('stock','s_w_id'); (1 row) BEGIN; -SET client_min_messages TO DEBUG4; -SET citus.task_executor_type TO 'task-tracker'; +SET client_min_messages TO DEBUG; -- Debug4 log messages display jobIds within them. We explicitly set the jobId -- sequence here so that the regression output becomes independent of the number -- of jobs executed prior to running this test. @@ -65,62 +65,26 @@ GROUP BY ORDER BY l_partkey, o_orderkey; DEBUG: Router planner does not support append-partitioned tables. -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: no valid constraints found -DEBUG: shard count: 2 DEBUG: join prunable for intervals [1,5986] and [8997,14947] DEBUG: join prunable for intervals [8997,14947] and [1,5986] -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_290000 lineitem JOIN orders_290002 orders ON ((lineitem.l_orderkey OPERATOR(pg_catalog.=) orders.o_orderkey))) WHERE ((lineitem.l_partkey OPERATOR(pg_catalog.<) 1000) AND (orders.o_totalprice OPERATOR(pg_catalog.>) 10.0))" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_290001 lineitem JOIN orders_290003 orders ON ((lineitem.l_orderkey OPERATOR(pg_catalog.=) orders.o_orderkey))) WHERE ((lineitem.l_partkey OPERATOR(pg_catalog.<) 1000) AND (orders.o_totalprice OPERATOR(pg_catalog.>) 10.0))" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 2 DEBUG: join prunable for intervals [1,1000] and [6001,7000] DEBUG: join prunable for intervals [6001,7000] and [1,1000] -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT "pg_merge_job_0001.task_000003".intermediate_column_1_0, "pg_merge_job_0001.task_000003".intermediate_column_1_1, "pg_merge_job_0001.task_000003".intermediate_column_1_2, "pg_merge_job_0001.task_000003".intermediate_column_1_3, "pg_merge_job_0001.task_000003".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000003 "pg_merge_job_0001.task_000003" JOIN part_append_290005 part_append ON (("pg_merge_job_0001.task_000003".intermediate_column_1_0 OPERATOR(pg_catalog.=) part_append.p_partkey))) WHERE (part_append.p_size OPERATOR(pg_catalog.>) 8)" -DEBUG: generated sql query for task 4 -DETAIL: query string: "SELECT "pg_merge_job_0001.task_000006".intermediate_column_1_0, "pg_merge_job_0001.task_000006".intermediate_column_1_1, "pg_merge_job_0001.task_000006".intermediate_column_1_2, "pg_merge_job_0001.task_000006".intermediate_column_1_3, "pg_merge_job_0001.task_000006".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000006 "pg_merge_job_0001.task_000006" JOIN part_append_280002 part_append ON (("pg_merge_job_0001.task_000006".intermediate_column_1_0 OPERATOR(pg_catalog.=) part_append.p_partkey))) WHERE (part_append.p_size OPERATOR(pg_catalog.>) 8)" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 3 DEBUG: pruning merge fetch taskId 3 DETAIL: Creating dependency on merge taskId 6 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 3 DEBUG: join prunable for intervals [1,1000] and [1001,2000] DEBUG: join prunable for intervals [1,1000] and [6001,7000] DEBUG: join prunable for intervals [1001,2000] and [1,1000] DEBUG: join prunable for intervals [1001,2000] and [6001,7000] DEBUG: join prunable for intervals [6001,7000] and [1,1000] DEBUG: join prunable for intervals [6001,7000] and [1001,2000] -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT "pg_merge_job_0002.task_000005".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000005".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000005 "pg_merge_job_0002.task_000005" JOIN customer_append_290004 customer_append ON ((customer_append.c_custkey OPERATOR(pg_catalog.=) "pg_merge_job_0002.task_000005".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000005".intermediate_column_2_2 OPERATOR(pg_catalog.>) 5.0) OR ("pg_merge_job_0002.task_000005".intermediate_column_2_3 OPERATOR(pg_catalog.>) 1200.0)) AND (customer_append.c_acctbal OPERATOR(pg_catalog.<) 5000.0)) GROUP BY "pg_merge_job_0002.task_000005".intermediate_column_2_0, "pg_merge_job_0002.task_000005".intermediate_column_2_1" -DEBUG: generated sql query for task 4 -DETAIL: query string: "SELECT "pg_merge_job_0002.task_000008".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000008".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000008 "pg_merge_job_0002.task_000008" JOIN customer_append_280001 customer_append ON ((customer_append.c_custkey OPERATOR(pg_catalog.=) "pg_merge_job_0002.task_000008".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000008".intermediate_column_2_2 OPERATOR(pg_catalog.>) 5.0) OR ("pg_merge_job_0002.task_000008".intermediate_column_2_3 OPERATOR(pg_catalog.>) 1200.0)) AND (customer_append.c_acctbal OPERATOR(pg_catalog.<) 5000.0)) GROUP BY "pg_merge_job_0002.task_000008".intermediate_column_2_0, "pg_merge_job_0002.task_000008".intermediate_column_2_1" -DEBUG: generated sql query for task 6 -DETAIL: query string: "SELECT "pg_merge_job_0002.task_000011".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000011".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000011 "pg_merge_job_0002.task_000011" JOIN customer_append_280000 customer_append ON ((customer_append.c_custkey OPERATOR(pg_catalog.=) "pg_merge_job_0002.task_000011".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000011".intermediate_column_2_2 OPERATOR(pg_catalog.>) 5.0) OR ("pg_merge_job_0002.task_000011".intermediate_column_2_3 OPERATOR(pg_catalog.>) 1200.0)) AND (customer_append.c_acctbal OPERATOR(pg_catalog.<) 5000.0)) GROUP BY "pg_merge_job_0002.task_000011".intermediate_column_2_0, "pg_merge_job_0002.task_000011".intermediate_column_2_1" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 5 DEBUG: pruning merge fetch taskId 3 DETAIL: Creating dependency on merge taskId 8 DEBUG: pruning merge fetch taskId 5 DETAIL: Creating dependency on merge taskId 11 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: combine query: SELECT l_partkey, o_orderkey, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 3 2 4)'::cstring(0)) remote_scan(l_partkey integer, o_orderkey bigint, count bigint) GROUP BY l_partkey, o_orderkey ORDER BY l_partkey, o_orderkey -DEBUG: completed cleanup query for job 3 -DEBUG: completed cleanup query for job 3 -DEBUG: completed cleanup query for job 2 -DEBUG: completed cleanup query for job 2 -DEBUG: completed cleanup query for job 1 -DEBUG: completed cleanup query for job 1 l_partkey | o_orderkey | count --------------------------------------------------------------------- 18 | 12005 | 1 @@ -166,22 +130,6 @@ GROUP BY ORDER BY l_partkey, o_orderkey; DEBUG: Router planner does not support append-partitioned tables. -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_290000 lineitem WHERE (l_quantity OPERATOR(pg_catalog.<) 5.0)" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_290001 lineitem WHERE (l_quantity OPERATOR(pg_catalog.<) 5.0)" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290002 orders WHERE (o_totalprice OPERATOR(pg_catalog.<>) 4.0)" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290003 orders WHERE (o_totalprice OPERATOR(pg_catalog.<>) 4.0)" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx DEBUG: join prunable for task partitionId 0 and 1 DEBUG: join prunable for task partitionId 0 and 2 DEBUG: join prunable for task partitionId 0 and 3 @@ -194,14 +142,6 @@ DEBUG: join prunable for task partitionId 2 and 3 DEBUG: join prunable for task partitionId 3 and 0 DEBUG: join prunable for task partitionId 3 and 1 DEBUG: join prunable for task partitionId 3 and 2 -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT "pg_merge_job_0004.task_000003".intermediate_column_4_0 AS l_partkey, "pg_merge_job_0005.task_000003".intermediate_column_5_0 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0004.task_000003 "pg_merge_job_0004.task_000003" JOIN pg_merge_job_0005.task_000003 "pg_merge_job_0005.task_000003" ON (("pg_merge_job_0004.task_000003".intermediate_column_4_1 OPERATOR(pg_catalog.=) "pg_merge_job_0005.task_000003".intermediate_column_5_1))) WHERE true GROUP BY "pg_merge_job_0004.task_000003".intermediate_column_4_0, "pg_merge_job_0005.task_000003".intermediate_column_5_0" -DEBUG: generated sql query for task 6 -DETAIL: query string: "SELECT "pg_merge_job_0004.task_000006".intermediate_column_4_0 AS l_partkey, "pg_merge_job_0005.task_000006".intermediate_column_5_0 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0004.task_000006 "pg_merge_job_0004.task_000006" JOIN pg_merge_job_0005.task_000006 "pg_merge_job_0005.task_000006" ON (("pg_merge_job_0004.task_000006".intermediate_column_4_1 OPERATOR(pg_catalog.=) "pg_merge_job_0005.task_000006".intermediate_column_5_1))) WHERE true GROUP BY "pg_merge_job_0004.task_000006".intermediate_column_4_0, "pg_merge_job_0005.task_000006".intermediate_column_5_0" -DEBUG: generated sql query for task 9 -DETAIL: query string: "SELECT "pg_merge_job_0004.task_000009".intermediate_column_4_0 AS l_partkey, "pg_merge_job_0005.task_000009".intermediate_column_5_0 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0004.task_000009 "pg_merge_job_0004.task_000009" JOIN pg_merge_job_0005.task_000009 "pg_merge_job_0005.task_000009" ON (("pg_merge_job_0004.task_000009".intermediate_column_4_1 OPERATOR(pg_catalog.=) "pg_merge_job_0005.task_000009".intermediate_column_5_1))) WHERE true GROUP BY "pg_merge_job_0004.task_000009".intermediate_column_4_0, "pg_merge_job_0005.task_000009".intermediate_column_5_0" -DEBUG: generated sql query for task 12 -DETAIL: query string: "SELECT "pg_merge_job_0004.task_000012".intermediate_column_4_0 AS l_partkey, "pg_merge_job_0005.task_000012".intermediate_column_5_0 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0004.task_000012 "pg_merge_job_0004.task_000012" JOIN pg_merge_job_0005.task_000012 "pg_merge_job_0005.task_000012" ON (("pg_merge_job_0004.task_000012".intermediate_column_4_1 OPERATOR(pg_catalog.=) "pg_merge_job_0005.task_000012".intermediate_column_5_1))) WHERE true GROUP BY "pg_merge_job_0004.task_000012".intermediate_column_4_0, "pg_merge_job_0005.task_000012".intermediate_column_5_0" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 3 DEBUG: pruning merge fetch taskId 2 @@ -218,17 +158,6 @@ DEBUG: pruning merge fetch taskId 10 DETAIL: Creating dependency on merge taskId 12 DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 12 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: combine query: SELECT l_partkey, o_orderkey, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(l_partkey integer, o_orderkey bigint, count bigint) GROUP BY l_partkey, o_orderkey ORDER BY l_partkey, o_orderkey -DEBUG: completed cleanup query for job 6 -DEBUG: completed cleanup query for job 6 -DEBUG: completed cleanup query for job 4 -DEBUG: completed cleanup query for job 4 -DEBUG: completed cleanup query for job 5 -DEBUG: completed cleanup query for job 5 l_partkey | o_orderkey | count --------------------------------------------------------------------- (0 rows) @@ -245,22 +174,6 @@ GROUP BY ORDER BY o_orderkey; DEBUG: Router planner does not support append-partitioned tables. -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT l_suppkey FROM lineitem_290000 lineitem WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT l_suppkey FROM lineitem_290001 lineitem WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290002 orders WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290003 orders WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx DEBUG: join prunable for task partitionId 0 and 1 DEBUG: join prunable for task partitionId 0 and 2 DEBUG: join prunable for task partitionId 0 and 3 @@ -273,14 +186,6 @@ DEBUG: join prunable for task partitionId 2 and 3 DEBUG: join prunable for task partitionId 3 and 0 DEBUG: join prunable for task partitionId 3 and 1 DEBUG: join prunable for task partitionId 3 and 2 -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT "pg_merge_job_0008.task_000003".intermediate_column_8_0 AS o_orderkey, any_value("pg_merge_job_0008.task_000003".intermediate_column_8_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0007.task_000003 "pg_merge_job_0007.task_000003" JOIN pg_merge_job_0008.task_000003 "pg_merge_job_0008.task_000003" ON (("pg_merge_job_0007.task_000003".intermediate_column_7_0 OPERATOR(pg_catalog.=) "pg_merge_job_0008.task_000003".intermediate_column_8_1))) WHERE true GROUP BY "pg_merge_job_0008.task_000003".intermediate_column_8_0" -DEBUG: generated sql query for task 6 -DETAIL: query string: "SELECT "pg_merge_job_0008.task_000006".intermediate_column_8_0 AS o_orderkey, any_value("pg_merge_job_0008.task_000006".intermediate_column_8_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0007.task_000006 "pg_merge_job_0007.task_000006" JOIN pg_merge_job_0008.task_000006 "pg_merge_job_0008.task_000006" ON (("pg_merge_job_0007.task_000006".intermediate_column_7_0 OPERATOR(pg_catalog.=) "pg_merge_job_0008.task_000006".intermediate_column_8_1))) WHERE true GROUP BY "pg_merge_job_0008.task_000006".intermediate_column_8_0" -DEBUG: generated sql query for task 9 -DETAIL: query string: "SELECT "pg_merge_job_0008.task_000009".intermediate_column_8_0 AS o_orderkey, any_value("pg_merge_job_0008.task_000009".intermediate_column_8_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0007.task_000009 "pg_merge_job_0007.task_000009" JOIN pg_merge_job_0008.task_000009 "pg_merge_job_0008.task_000009" ON (("pg_merge_job_0007.task_000009".intermediate_column_7_0 OPERATOR(pg_catalog.=) "pg_merge_job_0008.task_000009".intermediate_column_8_1))) WHERE true GROUP BY "pg_merge_job_0008.task_000009".intermediate_column_8_0" -DEBUG: generated sql query for task 12 -DETAIL: query string: "SELECT "pg_merge_job_0008.task_000012".intermediate_column_8_0 AS o_orderkey, any_value("pg_merge_job_0008.task_000012".intermediate_column_8_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0007.task_000012 "pg_merge_job_0007.task_000012" JOIN pg_merge_job_0008.task_000012 "pg_merge_job_0008.task_000012" ON (("pg_merge_job_0007.task_000012".intermediate_column_7_0 OPERATOR(pg_catalog.=) "pg_merge_job_0008.task_000012".intermediate_column_8_1))) WHERE true GROUP BY "pg_merge_job_0008.task_000012".intermediate_column_8_0" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 3 DEBUG: pruning merge fetch taskId 2 @@ -297,17 +202,6 @@ DEBUG: pruning merge fetch taskId 10 DETAIL: Creating dependency on merge taskId 12 DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 12 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: combine query: SELECT o_orderkey, o_shippriority, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, o_shippriority integer, count bigint) GROUP BY o_orderkey ORDER BY o_orderkey -DEBUG: completed cleanup query for job 9 -DEBUG: completed cleanup query for job 9 -DEBUG: completed cleanup query for job 7 -DEBUG: completed cleanup query for job 7 -DEBUG: completed cleanup query for job 8 -DEBUG: completed cleanup query for job 8 o_orderkey | o_shippriority | count --------------------------------------------------------------------- (0 rows) @@ -326,22 +220,6 @@ GROUP BY ORDER BY o_orderkey; DEBUG: Router planner does not support append-partitioned tables. -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT l_suppkey FROM lineitem_290000 lineitem WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT l_suppkey FROM lineitem_290001 lineitem WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290002 orders WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290003 orders WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx DEBUG: join prunable for task partitionId 0 and 1 DEBUG: join prunable for task partitionId 0 and 2 DEBUG: join prunable for task partitionId 0 and 3 @@ -354,14 +232,6 @@ DEBUG: join prunable for task partitionId 2 and 3 DEBUG: join prunable for task partitionId 3 and 0 DEBUG: join prunable for task partitionId 3 and 1 DEBUG: join prunable for task partitionId 3 and 2 -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT "pg_merge_job_0011.task_000003".intermediate_column_11_0 AS o_orderkey, any_value("pg_merge_job_0011.task_000003".intermediate_column_11_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0010.task_000003 "pg_merge_job_0010.task_000003" JOIN pg_merge_job_0011.task_000003 "pg_merge_job_0011.task_000003" ON (("pg_merge_job_0010.task_000003".intermediate_column_10_0 OPERATOR(pg_catalog.=) "pg_merge_job_0011.task_000003".intermediate_column_11_1))) WHERE true GROUP BY "pg_merge_job_0011.task_000003".intermediate_column_11_0" -DEBUG: generated sql query for task 6 -DETAIL: query string: "SELECT "pg_merge_job_0011.task_000006".intermediate_column_11_0 AS o_orderkey, any_value("pg_merge_job_0011.task_000006".intermediate_column_11_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0010.task_000006 "pg_merge_job_0010.task_000006" JOIN pg_merge_job_0011.task_000006 "pg_merge_job_0011.task_000006" ON (("pg_merge_job_0010.task_000006".intermediate_column_10_0 OPERATOR(pg_catalog.=) "pg_merge_job_0011.task_000006".intermediate_column_11_1))) WHERE true GROUP BY "pg_merge_job_0011.task_000006".intermediate_column_11_0" -DEBUG: generated sql query for task 9 -DETAIL: query string: "SELECT "pg_merge_job_0011.task_000009".intermediate_column_11_0 AS o_orderkey, any_value("pg_merge_job_0011.task_000009".intermediate_column_11_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0010.task_000009 "pg_merge_job_0010.task_000009" JOIN pg_merge_job_0011.task_000009 "pg_merge_job_0011.task_000009" ON (("pg_merge_job_0010.task_000009".intermediate_column_10_0 OPERATOR(pg_catalog.=) "pg_merge_job_0011.task_000009".intermediate_column_11_1))) WHERE true GROUP BY "pg_merge_job_0011.task_000009".intermediate_column_11_0" -DEBUG: generated sql query for task 12 -DETAIL: query string: "SELECT "pg_merge_job_0011.task_000012".intermediate_column_11_0 AS o_orderkey, any_value("pg_merge_job_0011.task_000012".intermediate_column_11_1) AS o_shippriority, count(*) AS count FROM (pg_merge_job_0010.task_000012 "pg_merge_job_0010.task_000012" JOIN pg_merge_job_0011.task_000012 "pg_merge_job_0011.task_000012" ON (("pg_merge_job_0010.task_000012".intermediate_column_10_0 OPERATOR(pg_catalog.=) "pg_merge_job_0011.task_000012".intermediate_column_11_1))) WHERE true GROUP BY "pg_merge_job_0011.task_000012".intermediate_column_11_0" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 3 DEBUG: pruning merge fetch taskId 2 @@ -378,17 +248,6 @@ DEBUG: pruning merge fetch taskId 10 DETAIL: Creating dependency on merge taskId 12 DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 12 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: combine query: SELECT o_orderkey, o_shippriority, COALESCE((pg_catalog.sum(count))::bigint, '0'::bigint) AS count FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, o_shippriority integer, count bigint) GROUP BY o_orderkey ORDER BY o_orderkey -DEBUG: completed cleanup query for job 12 -DEBUG: completed cleanup query for job 12 -DEBUG: completed cleanup query for job 10 -DEBUG: completed cleanup query for job 10 -DEBUG: completed cleanup query for job 11 -DEBUG: completed cleanup query for job 11 o_orderkey | o_shippriority | count --------------------------------------------------------------------- (0 rows) @@ -405,22 +264,6 @@ GROUP BY ORDER BY o_orderkey; DEBUG: Router planner does not support append-partitioned tables. -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT l_suppkey FROM lineitem_290000 lineitem WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT l_suppkey FROM lineitem_290001 lineitem WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 2 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290002 orders WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_290003 orders WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx DEBUG: join prunable for task partitionId 0 and 1 DEBUG: join prunable for task partitionId 0 and 2 DEBUG: join prunable for task partitionId 0 and 3 @@ -433,14 +276,6 @@ DEBUG: join prunable for task partitionId 2 and 3 DEBUG: join prunable for task partitionId 3 and 0 DEBUG: join prunable for task partitionId 3 and 1 DEBUG: join prunable for task partitionId 3 and 2 -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT "pg_merge_job_0014.task_000003".intermediate_column_14_0 AS o_orderkey, any_value("pg_merge_job_0014.task_000003".intermediate_column_14_1) AS any_value, any_value("pg_merge_job_0014.task_000003".intermediate_column_14_1) AS worker_column_3 FROM (pg_merge_job_0013.task_000003 "pg_merge_job_0013.task_000003" JOIN pg_merge_job_0014.task_000003 "pg_merge_job_0014.task_000003" ON (("pg_merge_job_0013.task_000003".intermediate_column_13_0 OPERATOR(pg_catalog.=) "pg_merge_job_0014.task_000003".intermediate_column_14_1))) WHERE true GROUP BY "pg_merge_job_0014.task_000003".intermediate_column_14_0" -DEBUG: generated sql query for task 6 -DETAIL: query string: "SELECT "pg_merge_job_0014.task_000006".intermediate_column_14_0 AS o_orderkey, any_value("pg_merge_job_0014.task_000006".intermediate_column_14_1) AS any_value, any_value("pg_merge_job_0014.task_000006".intermediate_column_14_1) AS worker_column_3 FROM (pg_merge_job_0013.task_000006 "pg_merge_job_0013.task_000006" JOIN pg_merge_job_0014.task_000006 "pg_merge_job_0014.task_000006" ON (("pg_merge_job_0013.task_000006".intermediate_column_13_0 OPERATOR(pg_catalog.=) "pg_merge_job_0014.task_000006".intermediate_column_14_1))) WHERE true GROUP BY "pg_merge_job_0014.task_000006".intermediate_column_14_0" -DEBUG: generated sql query for task 9 -DETAIL: query string: "SELECT "pg_merge_job_0014.task_000009".intermediate_column_14_0 AS o_orderkey, any_value("pg_merge_job_0014.task_000009".intermediate_column_14_1) AS any_value, any_value("pg_merge_job_0014.task_000009".intermediate_column_14_1) AS worker_column_3 FROM (pg_merge_job_0013.task_000009 "pg_merge_job_0013.task_000009" JOIN pg_merge_job_0014.task_000009 "pg_merge_job_0014.task_000009" ON (("pg_merge_job_0013.task_000009".intermediate_column_13_0 OPERATOR(pg_catalog.=) "pg_merge_job_0014.task_000009".intermediate_column_14_1))) WHERE true GROUP BY "pg_merge_job_0014.task_000009".intermediate_column_14_0" -DEBUG: generated sql query for task 12 -DETAIL: query string: "SELECT "pg_merge_job_0014.task_000012".intermediate_column_14_0 AS o_orderkey, any_value("pg_merge_job_0014.task_000012".intermediate_column_14_1) AS any_value, any_value("pg_merge_job_0014.task_000012".intermediate_column_14_1) AS worker_column_3 FROM (pg_merge_job_0013.task_000012 "pg_merge_job_0013.task_000012" JOIN pg_merge_job_0014.task_000012 "pg_merge_job_0014.task_000012" ON (("pg_merge_job_0013.task_000012".intermediate_column_13_0 OPERATOR(pg_catalog.=) "pg_merge_job_0014.task_000012".intermediate_column_14_1))) WHERE true GROUP BY "pg_merge_job_0014.task_000012".intermediate_column_14_0" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 3 DEBUG: pruning merge fetch taskId 2 @@ -457,17 +292,6 @@ DEBUG: pruning merge fetch taskId 10 DETAIL: Creating dependency on merge taskId 12 DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 12 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: combine query: SELECT o_orderkey, any_value(any_value) AS any_value FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(o_orderkey bigint, any_value integer, worker_column_3 integer) GROUP BY o_orderkey ORDER BY o_orderkey -DEBUG: completed cleanup query for job 15 -DEBUG: completed cleanup query for job 15 -DEBUG: completed cleanup query for job 13 -DEBUG: completed cleanup query for job 13 -DEBUG: completed cleanup query for job 14 -DEBUG: completed cleanup query for job 14 o_orderkey | any_value --------------------------------------------------------------------- (0 rows) @@ -481,39 +305,7 @@ select s_i_id group by s_i_id, s_w_id, s_quantity having s_quantity > random() ; -DEBUG: no valid constraints found -DEBUG: shard count: 4 -DEBUG: no valid constraints found -DEBUG: shard count: 4 DEBUG: Router planner cannot handle multi-shard select queries -DEBUG: no valid constraints found -DEBUG: shard count: 4 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT s_i_id, s_w_id, s_quantity FROM stock_690004 stock WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT s_i_id, s_w_id, s_quantity FROM stock_690005 stock WHERE true" -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT s_i_id, s_w_id, s_quantity FROM stock_690006 stock WHERE true" -DEBUG: generated sql query for task 4 -DETAIL: query string: "SELECT s_i_id, s_w_id, s_quantity FROM stock_690007 stock WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: no valid constraints found -DEBUG: shard count: 4 -DEBUG: generated sql query for task 1 -DETAIL: query string: "SELECT ol_i_id FROM order_line_690000 order_line WHERE true" -DEBUG: generated sql query for task 2 -DETAIL: query string: "SELECT ol_i_id FROM order_line_690001 order_line WHERE true" -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT ol_i_id FROM order_line_690002 order_line WHERE true" -DEBUG: generated sql query for task 4 -DETAIL: query string: "SELECT ol_i_id FROM order_line_690003 order_line WHERE true" -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx DEBUG: join prunable for task partitionId 0 and 1 DEBUG: join prunable for task partitionId 0 and 2 DEBUG: join prunable for task partitionId 0 and 3 @@ -526,14 +318,6 @@ DEBUG: join prunable for task partitionId 2 and 3 DEBUG: join prunable for task partitionId 3 and 0 DEBUG: join prunable for task partitionId 3 and 1 DEBUG: join prunable for task partitionId 3 and 2 -DEBUG: generated sql query for task 3 -DETAIL: query string: "SELECT "pg_merge_job_0016.task_000005".intermediate_column_16_0 AS s_i_id, "pg_merge_job_0016.task_000005".intermediate_column_16_1 AS worker_column_2, any_value("pg_merge_job_0016.task_000005".intermediate_column_16_2) AS worker_column_3 FROM (pg_merge_job_0016.task_000005 "pg_merge_job_0016.task_000005" JOIN pg_merge_job_0017.task_000005 "pg_merge_job_0017.task_000005" ON (("pg_merge_job_0017.task_000005".intermediate_column_17_0 OPERATOR(pg_catalog.=) "pg_merge_job_0016.task_000005".intermediate_column_16_0))) WHERE true GROUP BY "pg_merge_job_0016.task_000005".intermediate_column_16_0, "pg_merge_job_0016.task_000005".intermediate_column_16_1 HAVING ((any_value("pg_merge_job_0016.task_000005".intermediate_column_16_2))::double precision OPERATOR(pg_catalog.>) random())" -DEBUG: generated sql query for task 6 -DETAIL: query string: "SELECT "pg_merge_job_0016.task_000010".intermediate_column_16_0 AS s_i_id, "pg_merge_job_0016.task_000010".intermediate_column_16_1 AS worker_column_2, any_value("pg_merge_job_0016.task_000010".intermediate_column_16_2) AS worker_column_3 FROM (pg_merge_job_0016.task_000010 "pg_merge_job_0016.task_000010" JOIN pg_merge_job_0017.task_000010 "pg_merge_job_0017.task_000010" ON (("pg_merge_job_0017.task_000010".intermediate_column_17_0 OPERATOR(pg_catalog.=) "pg_merge_job_0016.task_000010".intermediate_column_16_0))) WHERE true GROUP BY "pg_merge_job_0016.task_000010".intermediate_column_16_0, "pg_merge_job_0016.task_000010".intermediate_column_16_1 HAVING ((any_value("pg_merge_job_0016.task_000010".intermediate_column_16_2))::double precision OPERATOR(pg_catalog.>) random())" -DEBUG: generated sql query for task 9 -DETAIL: query string: "SELECT "pg_merge_job_0016.task_000015".intermediate_column_16_0 AS s_i_id, "pg_merge_job_0016.task_000015".intermediate_column_16_1 AS worker_column_2, any_value("pg_merge_job_0016.task_000015".intermediate_column_16_2) AS worker_column_3 FROM (pg_merge_job_0016.task_000015 "pg_merge_job_0016.task_000015" JOIN pg_merge_job_0017.task_000015 "pg_merge_job_0017.task_000015" ON (("pg_merge_job_0017.task_000015".intermediate_column_17_0 OPERATOR(pg_catalog.=) "pg_merge_job_0016.task_000015".intermediate_column_16_0))) WHERE true GROUP BY "pg_merge_job_0016.task_000015".intermediate_column_16_0, "pg_merge_job_0016.task_000015".intermediate_column_16_1 HAVING ((any_value("pg_merge_job_0016.task_000015".intermediate_column_16_2))::double precision OPERATOR(pg_catalog.>) random())" -DEBUG: generated sql query for task 12 -DETAIL: query string: "SELECT "pg_merge_job_0016.task_000020".intermediate_column_16_0 AS s_i_id, "pg_merge_job_0016.task_000020".intermediate_column_16_1 AS worker_column_2, any_value("pg_merge_job_0016.task_000020".intermediate_column_16_2) AS worker_column_3 FROM (pg_merge_job_0016.task_000020 "pg_merge_job_0016.task_000020" JOIN pg_merge_job_0017.task_000020 "pg_merge_job_0017.task_000020" ON (("pg_merge_job_0017.task_000020".intermediate_column_17_0 OPERATOR(pg_catalog.=) "pg_merge_job_0016.task_000020".intermediate_column_16_0))) WHERE true GROUP BY "pg_merge_job_0016.task_000020".intermediate_column_16_0, "pg_merge_job_0016.task_000020".intermediate_column_16_1 HAVING ((any_value("pg_merge_job_0016.task_000020".intermediate_column_16_2))::double precision OPERATOR(pg_catalog.>) random())" DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 5 DEBUG: pruning merge fetch taskId 2 @@ -550,17 +334,6 @@ DEBUG: pruning merge fetch taskId 10 DETAIL: Creating dependency on merge taskId 20 DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 20 -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: assigned task to node localhost:xxxxx -DEBUG: combine query: SELECT s_i_id FROM pg_catalog.citus_extradata_container(XXX, NULL::cstring(0), NULL::cstring(0), '(i 1 2)'::cstring(0)) remote_scan(s_i_id integer, worker_column_2 integer, worker_column_3 numeric) -DEBUG: completed cleanup query for job 18 -DEBUG: completed cleanup query for job 18 -DEBUG: completed cleanup query for job 16 -DEBUG: completed cleanup query for job 16 -DEBUG: completed cleanup query for job 17 -DEBUG: completed cleanup query for job 17 s_i_id --------------------------------------------------------------------- (0 rows) diff --git a/src/test/regress/expected/multi_repartition_join_pruning.out b/src/test/regress/expected/multi_repartition_join_pruning.out index b32d88cd8..f0dced987 100644 --- a/src/test/regress/expected/multi_repartition_join_pruning.out +++ b/src/test/regress/expected/multi_repartition_join_pruning.out @@ -6,7 +6,7 @@ -- jobs with real time executor. SET citus.next_shard_id TO 700000; SET client_min_messages TO DEBUG2; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; -- Single range-repartition join to test join-pruning behaviour. EXPLAIN (COSTS OFF) SELECT @@ -31,7 +31,7 @@ DETAIL: Creating dependency on merge taskId 9 QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 3 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -77,7 +77,7 @@ DEBUG: Router planner does not support append-partitioned tables. QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -112,7 +112,7 @@ DEBUG: Router planner does not support append-partitioned tables. QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -175,7 +175,7 @@ DETAIL: Creating dependency on merge taskId 16 QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -240,7 +240,7 @@ DEBUG: Router planner does not support append-partitioned tables. QUERY PLAN --------------------------------------------------------------------- Aggregate - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -275,7 +275,7 @@ WHERE DEBUG: Router planner does not support append-partitioned tables. QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -305,7 +305,7 @@ WHERE DEBUG: Router planner does not support append-partitioned tables. QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob @@ -319,9 +319,9 @@ SELECT FROM orders INNER JOIN customer_append ON (o_custkey = c_custkey AND false); DEBUG: Router planner does not support append-partitioned tables. - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: All (3 rows) @@ -334,9 +334,9 @@ FROM WHERE o_custkey = c_custkey AND false; DEBUG: Router planner does not support append-partitioned tables. - QUERY PLAN + QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) Task Count: 0 Tasks Shown: All (3 rows) diff --git a/src/test/regress/expected/multi_repartition_join_task_assignment.out b/src/test/regress/expected/multi_repartition_join_task_assignment.out index 78d0216a8..7d00a08c1 100644 --- a/src/test/regress/expected/multi_repartition_join_task_assignment.out +++ b/src/test/regress/expected/multi_repartition_join_task_assignment.out @@ -8,7 +8,7 @@ SET citus.next_shard_id TO 710000; BEGIN; SET client_min_messages TO DEBUG3; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; -- Single range repartition join to test anchor-shard based task assignment and -- assignment propagation to merge and data-fetch tasks. SELECT diff --git a/src/test/regress/expected/multi_repartition_udt.out b/src/test/regress/expected/multi_repartition_udt.out index f904a1029..4724ef887 100644 --- a/src/test/regress/expected/multi_repartition_udt.out +++ b/src/test/regress/expected/multi_repartition_udt.out @@ -2,6 +2,7 @@ -- MULTI_REPARTITION_UDT -- SET citus.next_shard_id TO 535000; +SET citus.enable_repartition_joins to ON; -- START type creation CREATE TYPE test_udt AS (i integer, i2 integer); -- ... as well as a function to use as its comparator... @@ -125,6 +126,7 @@ FUNCTION 1 test_udt_hash(test_udt); -- Distribute and populate the two tables. SET citus.shard_count TO 3; SET citus.shard_replication_factor TO 1; +SET citus.enable_repartition_joins to ON; SELECT create_distributed_table('repartition_udt', 'pk', 'hash'); create_distributed_table --------------------------------------------------------------------- @@ -171,7 +173,7 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ] QUERY PLAN --------------------------------------------------------------------- - Custom Scan (Citus Task-Tracker) + Custom Scan (Citus Adaptive) Task Count: 4 Tasks Shown: None, not supported for re-partition queries -> MapMergeJob diff --git a/src/test/regress/expected/multi_repartitioned_subquery_udf.out b/src/test/regress/expected/multi_repartitioned_subquery_udf.out index ae4b140af..0048bb7eb 100644 --- a/src/test/regress/expected/multi_repartitioned_subquery_udf.out +++ b/src/test/regress/expected/multi_repartitioned_subquery_udf.out @@ -35,7 +35,6 @@ LANGUAGE sql IMMUTABLE AS $_$ $_$; -- Run query on master \c - - :master_host :master_port -SET citus.task_executor_type TO 'task-tracker'; SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*) FROM lineitem GROUP BY l_partkey) AS a WHERE median > 2; diff --git a/src/test/regress/expected/multi_router_planner.out b/src/test/regress/expected/multi_router_planner.out index 6de5db827..c2d80d773 100644 --- a/src/test/regress/expected/multi_router_planner.out +++ b/src/test/regress/expected/multi_router_planner.out @@ -123,7 +123,6 @@ INSERT INTO articles_hash VALUES (47, 7, 'abeyance', 1772); INSERT INTO articles_hash VALUES (48, 8, 'alkylic', 18610); INSERT INTO articles_hash VALUES (49, 9, 'anyone', 2681); INSERT INTO articles_hash VALUES (50, 10, 'anjanette', 19519); -RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- insert a single row for the test INSERT INTO articles_single_shard_hash VALUES (50, 10, 'anjanette', 19519); @@ -1107,7 +1106,6 @@ ORDER BY 1,2,3,4; 43 | 3 | affixal | 12723 (15 rows) -RESET citus.task_executor_type; -- Test various filtering options for router plannable check SET client_min_messages to 'DEBUG2'; -- this is definitely single shard @@ -1774,17 +1772,17 @@ SELECT master_create_distributed_table('articles_range', 'author_id', 'range'); SELECT master_create_empty_shard('authors_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 1, shardmaxvalue=10 WHERE shardid = :shard_id; SELECT master_create_empty_shard('authors_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=30 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=20 WHERE shardid = :shard_id; SELECT master_create_empty_shard('authors_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=40 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=30 WHERE shardid = :shard_id; SELECT master_create_empty_shard('authors_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 31, shardmaxvalue=40 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 1, shardmaxvalue=10 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=30 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=20 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=40 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=30 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 31, shardmaxvalue=40 WHERE shardid = :shard_id; SET citus.log_remote_commands TO on; @@ -1840,7 +1838,7 @@ NOTICE: executing the command locally: SELECT ar.id, ar.author_id, ar.title, ar RESET citus.log_remote_commands; -- This query was intended to test "multi-shard join is not router plannable" -- To run it using repartition join logic we change the join columns -SET citus.task_executor_type to "task-tracker"; +SET citus.enable_repartition_joins to ON; SELECT * FROM articles_range ar join authors_range au on (ar.title = au.name) WHERE ar.author_id = 35; DEBUG: Router planner cannot handle multi-shard select queries @@ -1857,19 +1855,19 @@ DEBUG: join prunable for task partitionId 3 and 0 DEBUG: join prunable for task partitionId 3 and 1 DEBUG: join prunable for task partitionId 3 and 2 DEBUG: pruning merge fetch taskId 1 -DETAIL: Creating dependency on merge taskId 3 +DETAIL: Creating dependency on merge taskId 2 DEBUG: pruning merge fetch taskId 2 DETAIL: Creating dependency on merge taskId 5 DEBUG: pruning merge fetch taskId 4 -DETAIL: Creating dependency on merge taskId 6 +DETAIL: Creating dependency on merge taskId 4 DEBUG: pruning merge fetch taskId 5 DETAIL: Creating dependency on merge taskId 10 DEBUG: pruning merge fetch taskId 7 -DETAIL: Creating dependency on merge taskId 9 +DETAIL: Creating dependency on merge taskId 6 DEBUG: pruning merge fetch taskId 8 DETAIL: Creating dependency on merge taskId 15 DEBUG: pruning merge fetch taskId 10 -DETAIL: Creating dependency on merge taskId 12 +DETAIL: Creating dependency on merge taskId 8 DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 20 id | author_id | title | word_count | name | id @@ -1914,7 +1912,6 @@ DETAIL: Creating dependency on merge taskId 20 --------------------------------------------------------------------- (0 rows) -RESET citus.task_executor_type; -- bogus query, join on non-partition column, but router plannable due to filters SELECT * FROM articles_range ar join authors_range au on (ar.id = au.id) WHERE ar.author_id = 1 and au.id < 10; @@ -1941,14 +1938,18 @@ DETAIL: distribution column value: 2 SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id) WHERE ar.author_id = 3; DEBUG: found no worker with all shard placements -DEBUG: join prunable for intervals [1,10] and [11,30] -DEBUG: join prunable for intervals [1,10] and [21,40] +DEBUG: join prunable for intervals [1,10] and [11,20] +DEBUG: join prunable for intervals [1,10] and [21,30] DEBUG: join prunable for intervals [1,10] and [31,40] -DEBUG: join prunable for intervals [11,30] and [1,10] -DEBUG: join prunable for intervals [11,30] and [31,40] -DEBUG: join prunable for intervals [21,40] and [1,10] +DEBUG: join prunable for intervals [11,20] and [1,10] +DEBUG: join prunable for intervals [11,20] and [21,30] +DEBUG: join prunable for intervals [11,20] and [31,40] +DEBUG: join prunable for intervals [21,30] and [1,10] +DEBUG: join prunable for intervals [21,30] and [11,20] +DEBUG: join prunable for intervals [21,30] and [31,40] DEBUG: join prunable for intervals [31,40] and [1,10] -DEBUG: join prunable for intervals [31,40] and [11,30] +DEBUG: join prunable for intervals [31,40] and [11,20] +DEBUG: join prunable for intervals [31,40] and [21,30] DEBUG: pruning merge fetch taskId 1 DETAIL: Creating dependency on merge taskId 2 DEBUG: pruning merge fetch taskId 3 @@ -1956,17 +1957,11 @@ DETAIL: Creating dependency on merge taskId 4 DEBUG: pruning merge fetch taskId 5 DETAIL: Creating dependency on merge taskId 6 DEBUG: pruning merge fetch taskId 7 -DETAIL: Creating dependency on merge taskId 4 -DEBUG: pruning merge fetch taskId 9 -DETAIL: Creating dependency on merge taskId 6 -DEBUG: pruning merge fetch taskId 11 DETAIL: Creating dependency on merge taskId 8 -DEBUG: pruning merge fetch taskId 13 -DETAIL: Creating dependency on merge taskId 6 -DEBUG: pruning merge fetch taskId 15 -DETAIL: Creating dependency on merge taskId 8 -ERROR: the query contains a join that requires repartitioning -HINT: Set citus.enable_repartition_joins to on to enable repartitioning + id | author_id | title | word_count | name | id +--------------------------------------------------------------------- +(0 rows) + -- join between a range partitioned table and reference table is router plannable SELECT * FROM articles_range ar join authors_reference au on (ar.author_id = au.id) WHERE ar.author_id = 1; @@ -2404,7 +2399,6 @@ SELECT * FROM mv_articles_hash_data ORDER BY 1, 2, 3, 4; (10 rows) -- router planner/executor is now enabled for task-tracker executor -SET citus.task_executor_type to 'task-tracker'; SELECT id FROM articles_hash WHERE author_id = 1 diff --git a/src/test/regress/expected/multi_router_planner_fast_path.out b/src/test/regress/expected/multi_router_planner_fast_path.out index e527914f3..ad3fa68cc 100644 --- a/src/test/regress/expected/multi_router_planner_fast_path.out +++ b/src/test/regress/expected/multi_router_planner_fast_path.out @@ -62,7 +62,6 @@ INSERT INTO articles_hash VALUES (1, 1, 'arsenous', 9572), (2, 2, 'abducing', (40, 10, 'attemper', 14976),(41, 1, 'aznavour', 11814),(42, 2, 'ausable', 15885),(43, 3, 'affixal', 12723), (44, 4, 'anteport', 16793),(45, 5, 'afrasia', 864),(46, 6, 'atlanta', 17702),(47, 7, 'abeyance', 1772), (48, 8, 'alkylic', 18610),(49, 9, 'anyone', 2681),(50, 10, 'anjanette', 19519); -RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- test simple select for a single row SELECT * FROM articles_hash WHERE author_id = 10 AND id = 50; @@ -2118,7 +2117,6 @@ SELECT * FROM mv_articles_hash_empty; (5 rows) -- fast-path router planner/executor is enabled for task-tracker executor -SET citus.task_executor_type to 'task-tracker'; SELECT id FROM articles_hash WHERE author_id = 1; diff --git a/src/test/regress/expected/multi_schema_support.out b/src/test/regress/expected/multi_schema_support.out index dae406c50..8126d03ba 100644 --- a/src/test/regress/expected/multi_schema_support.out +++ b/src/test/regress/expected/multi_schema_support.out @@ -828,6 +828,7 @@ SELECT create_distributed_table('test_schema_support_join_2.nation_hash', 'n_nat (1 row) \copy test_schema_support_join_2.nation_hash FROM STDIN with delimiter '|'; +SET citus.enable_repartition_joins to ON; -- check when search_path is public, -- join of two tables which are in different schemas, -- join on partition column @@ -889,7 +890,6 @@ WHERE (1 row) -- single repartition joins -SET citus.task_executor_type TO "task-tracker"; -- check when search_path is public, -- join of two tables which are in different schemas, -- join on partition column and non-partition column @@ -982,7 +982,6 @@ WHERE (1 row) -- set task_executor back to adaptive -SET citus.task_executor_type TO "adaptive"; -- test ALTER TABLE SET SCHEMA SET search_path TO public; CREATE SCHEMA old_schema; @@ -1307,7 +1306,6 @@ SELECT sum(result::int) FROM run_command_on_shards('run_test_schema.test_table', (1 row) -- test capital letters on both table and schema names -SET citus.task_executor_type to "adaptive"; -- create schema with weird names CREATE SCHEMA "CiTuS.TeeN"; CREATE SCHEMA "CiTUS.TEEN2"; diff --git a/src/test/regress/expected/multi_simple_queries.out b/src/test/regress/expected/multi_simple_queries.out index 3d6ce652c..935a6f78e 100644 --- a/src/test/regress/expected/multi_simple_queries.out +++ b/src/test/regress/expected/multi_simple_queries.out @@ -412,7 +412,6 @@ SELECT o_orderstatus, sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders -- now, test the cases where Citus do or do not need to create -- the master queries SET client_min_messages TO 'DEBUG2'; -SET citus.task_executor_type TO 'adaptive'; -- start with the simple lookup query SELECT * FROM articles diff --git a/src/test/regress/expected/multi_simple_queries_0.out b/src/test/regress/expected/multi_simple_queries_0.out index 820eef768..d24ca2e1a 100644 --- a/src/test/regress/expected/multi_simple_queries_0.out +++ b/src/test/regress/expected/multi_simple_queries_0.out @@ -356,7 +356,6 @@ SELECT o_orderstatus, sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders -- now, test the cases where Citus do or do not need to create -- the master queries SET client_min_messages TO 'DEBUG2'; -SET citus.task_executor_type TO 'adaptive'; -- start with the simple lookup query SELECT * FROM articles diff --git a/src/test/regress/expected/multi_single_relation_subquery.out b/src/test/regress/expected/multi_single_relation_subquery.out index aa097a015..07561d078 100644 --- a/src/test/regress/expected/multi_single_relation_subquery.out +++ b/src/test/regress/expected/multi_single_relation_subquery.out @@ -3,7 +3,6 @@ -- -- This test checks that we are able to run selected set of distributed SQL subqueries. SET citus.next_shard_id TO 860000; -SET citus.task_executor_type TO 'task-tracker'; select number_sum, count(*) as total, @@ -198,8 +197,15 @@ group by l_suppkey ORDER BY 2 DESC, 1 DESC LIMIT 5; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries with limit are not supported yet + l_suppkey | total_suppkey_count +--------------------------------------------------------------------- + 35 | 5 + 112 | 4 + 102 | 4 + 73 | 4 + 123 | 3 +(5 rows) + -- Check that we don't support subqueries without aggregates. select DISTINCT rounded_tax @@ -212,8 +218,11 @@ from l_tax) as distributed_table ORDER BY 1 DESC LIMIT 5; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries without aggregates are not supported yet + rounded_tax +--------------------------------------------------------------------- + 0 +(1 row) + -- Check that we support subqueries with count(distinct). select avg(different_shipment_days) diff --git a/src/test/regress/expected/multi_sql_function.out b/src/test/regress/expected/multi_sql_function.out index c762e827d..e1862c983 100644 --- a/src/test/regress/expected/multi_sql_function.out +++ b/src/test/regress/expected/multi_sql_function.out @@ -33,7 +33,6 @@ CREATE FUNCTION sql_test_no_4() RETURNS bigint AS ' o_custkey = c_custkey AND o_orderkey = l_orderkey; ' LANGUAGE SQL; -SET citus.task_executor_type TO 'task-tracker'; SET client_min_messages TO INFO; -- now, run plain SQL functions SELECT sql_test_no_1(); @@ -62,7 +61,6 @@ SELECT sql_test_no_4(); -- run the tests which do not require re-partition -- with real-time executor -RESET citus.task_executor_type; -- now, run plain SQL functions SELECT sql_test_no_1(); sql_test_no_1 diff --git a/src/test/regress/expected/multi_task_string_size.out b/src/test/regress/expected/multi_task_string_size.out index 7e4a3c188..7e8600c3c 100644 --- a/src/test/regress/expected/multi_task_string_size.out +++ b/src/test/regress/expected/multi_task_string_size.out @@ -211,7 +211,6 @@ SELECT create_distributed_table('wide_table', 'long_column_001'); (1 row) -SET citus.task_executor_type TO 'task-tracker'; SHOW citus.max_task_string_size; citus.max_task_string_size --------------------------------------------------------------------- @@ -228,8 +227,12 @@ SET client_min_messages to ERROR; SELECT raise_failed_execution(' SELECT u.* FROM wide_table u JOIN wide_table v ON (u.long_column_002 = v.long_column_003); '); -ERROR: Task failed to execute -CONTEXT: PL/pgSQL function raise_failed_execution(text) line 6 at RAISE + raise_failed_execution +--------------------------------------------------------------------- + +(1 row) + +SET citus.enable_repartition_joins to ON; -- following will succeed since it fetches few columns SELECT u.long_column_001, u.long_column_002, u.long_column_003 FROM wide_table u JOIN wide_table v ON (u.long_column_002 = v.long_column_003); long_column_001 | long_column_002 | long_column_003 @@ -239,4 +242,3 @@ SELECT u.long_column_001, u.long_column_002, u.long_column_003 FROM wide_table u RESET client_min_messages; DROP TABLE wide_table; RESET citus.shard_count; -RESET citus.task_executor_type; diff --git a/src/test/regress/expected/multi_view.out b/src/test/regress/expected/multi_view.out index 82e20b8f3..82942c5a2 100644 --- a/src/test/regress/expected/multi_view.out +++ b/src/test/regress/expected/multi_view.out @@ -198,14 +198,20 @@ SELECT count(*) FROM priority_orders JOIN air_shipped_lineitems ON (o_custkey = 192 (1 row) -SET citus.task_executor_type to "task-tracker"; -- single view repartition subqueries are not supported SELECT l_suppkey, count(*) FROM (SELECT l_suppkey, l_shipdate, count(*) FROM air_shipped_lineitems GROUP BY l_suppkey, l_shipdate) supps GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries without group by clause are not supported yet + l_suppkey | count +--------------------------------------------------------------------- + 7680 | 4 + 160 | 3 + 1042 | 3 + 1318 | 3 + 5873 | 3 +(5 rows) + -- logically same query without a view works fine SELECT l_suppkey, count(*) FROM (SELECT l_suppkey, l_shipdate, count(*) @@ -226,8 +232,15 @@ SELECT l_suppkey, count(*) FROM FROM (SELECT * FROM lineitem_hash_part WHERE l_shipmode = 'AIR') asi GROUP BY l_suppkey, l_shipdate) supps GROUP BY l_suppkey ORDER BY 2 DESC, 1 LIMIT 5; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries without group by clause are not supported yet + l_suppkey | count +--------------------------------------------------------------------- + 7680 | 4 + 160 | 3 + 1042 | 3 + 1318 | 3 + 5873 | 3 +(5 rows) + -- repartition query on view with single table subquery CREATE VIEW supp_count_view AS SELECT * FROM (SELECT l_suppkey, count(*) FROM lineitem_hash_part GROUP BY 1) s1; SELECT * FROM supp_count_view ORDER BY 2 DESC, 1 LIMIT 10; @@ -710,8 +723,8 @@ SET citus.subquery_pushdown to ON; -- still not supported since outer query does not have limit -- it shows a different (subquery with single relation) error message SELECT * FROM recent_10_users; -ERROR: cannot perform distributed planning on this query -DETAIL: Subqueries with limit are not supported yet +ERROR: cannot push down this subquery +DETAIL: Limit in subquery without limit in the outermost query is unsupported -- now it displays more correct error message SELECT et.* FROM recent_10_users JOIN events_table et USING(user_id); ERROR: cannot push down this subquery diff --git a/src/test/regress/expected/propagate_set_commands.out b/src/test/regress/expected/propagate_set_commands.out index 2a906162e..1f122c55d 100644 --- a/src/test/regress/expected/propagate_set_commands.out +++ b/src/test/regress/expected/propagate_set_commands.out @@ -11,7 +11,6 @@ INSERT INTO test VALUES (1,1), (3,3); -- test set local propagation SET citus.propagate_set_commands TO 'local'; -- make sure we send BEGIN before a SELECT -SET citus.task_executor_type TO 'adaptive'; SET citus.select_opens_transaction_block TO on; BEGIN; SELECT current_setting('enable_hashagg') FROM test WHERE id = 1; diff --git a/src/test/regress/expected/set_operation_and_local_tables.out b/src/test/regress/expected/set_operation_and_local_tables.out index e8242b032..0beeb7ba9 100644 --- a/src/test/regress/expected/set_operation_and_local_tables.out +++ b/src/test/regress/expected/set_operation_and_local_tables.out @@ -1,5 +1,6 @@ CREATE SCHEMA recursive_set_local; SET search_path TO recursive_set_local, public; +SET citus.enable_repartition_joins to ON; CREATE TABLE recursive_set_local.test (x int, y int); SELECT create_distributed_table('test', 'x'); create_distributed_table @@ -329,7 +330,6 @@ DEBUG: Plan is router executable 2 (2 rows) -SET citus.task_executor_type TO 'task-tracker'; -- repartition is recursively planned before the set operation (SELECT x FROM test) INTERSECT (SELECT t1.x FROM test as t1, test as t2 WHERE t1.x = t2.y LIMIT 2) INTERSECT (((SELECT x FROM local_test) UNION ALL (SELECT x FROM test)) INTERSECT (SELECT i FROM generate_series(0, 100) i)) ORDER BY 1 DESC; DEBUG: Local tables cannot be used in distributed queries. @@ -378,7 +378,6 @@ DEBUG: Plan is router executable 1 (2 rows) -SET citus.task_executor_type TO 'adaptive'; RESET client_min_messages; DROP SCHEMA recursive_set_local CASCADE; NOTICE: drop cascades to 3 other objects diff --git a/src/test/regress/expected/single_node.out b/src/test/regress/expected/single_node.out index e71c3ad79..355c4b3a1 100644 --- a/src/test/regress/expected/single_node.out +++ b/src/test/regress/expected/single_node.out @@ -278,7 +278,11 @@ SELECT count(*) FROM test; BEGIN; INSERT INTO ref(a, b) SELECT x, y FROM test; SELECT count(*) from ref; -ERROR: cannot switch local execution status from local execution required to local execution disabled since it can cause visibility problems in the current transaction + count +--------------------------------------------------------------------- + 8 +(1 row) + ROLLBACK; -- INSERT SELECT from distributed table to local table BEGIN; diff --git a/src/test/regress/expected/task_tracker_assign_task.out b/src/test/regress/expected/task_tracker_assign_task.out index 85085e848..e69de29bb 100644 --- a/src/test/regress/expected/task_tracker_assign_task.out +++ b/src/test/regress/expected/task_tracker_assign_task.out @@ -1,89 +0,0 @@ --- --- TASK_TRACKER_ASSIGN_TASK --- -\set JobId 401010 -\set SimpleTaskId 101101 -\set RecoverableTaskId 801102 -\set SimpleTaskTable lineitem_simple_task -\set BadQueryString '\'SELECT COUNT(*) FROM bad_table_name\'' -\set GoodQueryString '\'SELECT COUNT(*) FROM lineitem\'' -\set SelectAll 'SELECT *' --- We assign two tasks to the task tracker. The first task simply executes. The --- recoverable task on the other hand repeatedly fails, and we sleep until the --- task tracker stops retrying the recoverable task. -SELECT task_tracker_assign_task(:JobId, :SimpleTaskId, - 'COPY (SELECT * FROM lineitem) TO ' - '''base/pgsql_job_cache/job_401010/task_101101'''); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_assign_task(:JobId, :RecoverableTaskId, :BadQueryString); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - --- After assigning the two tasks, we wait for them to make progress. Note that --- these tasks get scheduled and run asynchronously, so if the sleep interval is --- not enough, the regression tests may fail on an overloaded box. -SELECT pg_sleep(3.0); - pg_sleep ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_task_status(:JobId, :SimpleTaskId); - task_tracker_task_status ---------------------------------------------------------------------- - 6 -(1 row) - -SELECT task_tracker_task_status(:JobId, :RecoverableTaskId); - task_tracker_task_status ---------------------------------------------------------------------- - 5 -(1 row) - -COPY :SimpleTaskTable FROM 'base/pgsql_job_cache/job_401010/task_101101'; -SELECT COUNT(*) FROM :SimpleTaskTable; - count ---------------------------------------------------------------------- - 12000 -(1 row) - -SELECT COUNT(*) AS diff_lhs FROM ( :SelectAll FROM :SimpleTaskTable EXCEPT ALL - :SelectAll FROM lineitem ) diff; - diff_lhs ---------------------------------------------------------------------- - 0 -(1 row) - -SELECT COUNT(*) As diff_rhs FROM ( :SelectAll FROM lineitem EXCEPT ALL - :SelectAll FROM :SimpleTaskTable ) diff; - diff_rhs ---------------------------------------------------------------------- - 0 -(1 row) - --- We now reassign the recoverable task with a good query string. This updates --- the task's query string, and reschedules the updated task for execution. -SELECT task_tracker_assign_task(:JobId, :RecoverableTaskId, :GoodQueryString); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -SELECT pg_sleep(2.0); - pg_sleep ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_task_status(:JobId, :RecoverableTaskId); - task_tracker_task_status ---------------------------------------------------------------------- - 6 -(1 row) - diff --git a/src/test/regress/expected/task_tracker_cleanup_job.out b/src/test/regress/expected/task_tracker_cleanup_job.out index d8156de13..e69de29bb 100644 --- a/src/test/regress/expected/task_tracker_cleanup_job.out +++ b/src/test/regress/expected/task_tracker_cleanup_job.out @@ -1,110 +0,0 @@ --- --- TASK_TRACKER_CLEANUP_JOB --- -SET citus.next_shard_id TO 1060000; -\set JobId 401010 -\set CompletedTaskId 801107 -\set RunningTaskId 801108 --- Test worker_cleanup_job_schema_cache -SELECT * FROM task_tracker_assign_task(2, 2, ''); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -SELECT count(*) FROM pg_catalog.pg_namespace WHERE nspname = 'pg_merge_job_0002'; - count ---------------------------------------------------------------------- - 1 -(1 row) - -SELECT worker_cleanup_job_schema_cache(); - worker_cleanup_job_schema_cache ---------------------------------------------------------------------- - -(1 row) - -SELECT count(*) FROM pg_catalog.pg_namespace WHERE nspname = 'pg_merge_job_0002'; - count ---------------------------------------------------------------------- - 0 -(1 row) - --- We assign two tasks to the task tracker. The first task should complete and --- the second task should continue to keep running. -SELECT task_tracker_assign_task(:JobId, :CompletedTaskId, - 'COPY (SELECT * FROM lineitem) TO ' - '''base/pgsql_job_cache/job_401010/task_801107'''); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_assign_task(:JobId, :RunningTaskId, - 'SELECT pg_sleep(100)'); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -SELECT pg_sleep(2.0); - pg_sleep ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_task_status(:JobId, :CompletedTaskId); - task_tracker_task_status ---------------------------------------------------------------------- - 6 -(1 row) - -SELECT task_tracker_task_status(:JobId, :RunningTaskId); - task_tracker_task_status ---------------------------------------------------------------------- - 3 -(1 row) - -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010/task_801107'); - isdir ---------------------------------------------------------------------- - f -(1 row) - -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010'); - isdir ---------------------------------------------------------------------- - t -(1 row) - --- We now clean up all tasks for this job id. As a result, shared hash entries, --- files, and connections associated with these tasks should all be cleaned up. -SELECT task_tracker_cleanup_job(:JobId); - task_tracker_cleanup_job ---------------------------------------------------------------------- - -(1 row) - -SELECT pg_sleep(1.0); - pg_sleep ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_task_status(:JobId, :CompletedTaskId); -ERROR: could not find the worker task -DETAIL: Task jobId: 401010 and taskId: 801107 -SELECT task_tracker_task_status(:JobId, :RunningTaskId); -ERROR: could not find the worker task -DETAIL: Task jobId: 401010 and taskId: 801108 -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010/task_801107'); -ERROR: could not stat file "base/pgsql_job_cache/job_401010/task_801107": No such file or directory -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010'); -ERROR: could not stat file "base/pgsql_job_cache/job_401010": No such file or directory --- Also clean up worker_cleanup_job_schema_cache job -SELECT task_tracker_cleanup_job(2); - task_tracker_cleanup_job ---------------------------------------------------------------------- - -(1 row) - diff --git a/src/test/regress/expected/task_tracker_create_table.out b/src/test/regress/expected/task_tracker_create_table.out index e34d79a98..e69de29bb 100644 --- a/src/test/regress/expected/task_tracker_create_table.out +++ b/src/test/regress/expected/task_tracker_create_table.out @@ -1,11 +0,0 @@ --- --- TASK_TRACKER_CREATE_TABLE --- -SET citus.next_shard_id TO 1070000; --- New table definitions to test the task tracker process and protocol -CREATE TABLE lineitem_simple_task ( LIKE lineitem ); -CREATE TABLE lineitem_compute_task ( LIKE lineitem ); -CREATE TABLE lineitem_compute_update_task ( LIKE lineitem ); -CREATE TABLE lineitem_partition_task_part_00 ( LIKE lineitem ); -CREATE TABLE lineitem_partition_task_part_01 ( LIKE lineitem ); -CREATE TABLE lineitem_partition_task_part_02 ( LIKE lineitem ); diff --git a/src/test/regress/expected/task_tracker_partition_task.out b/src/test/regress/expected/task_tracker_partition_task.out index 0bc0b1389..e69de29bb 100644 --- a/src/test/regress/expected/task_tracker_partition_task.out +++ b/src/test/regress/expected/task_tracker_partition_task.out @@ -1,108 +0,0 @@ --- --- TASK_TRACKER_PARTITION_TASK --- -\set JobId 401010 -\set PartitionTaskId 801106 -\set PartitionColumn l_orderkey -\set SelectAll 'SELECT *' -\set TablePart00 lineitem_partition_task_part_00 -\set TablePart01 lineitem_partition_task_part_01 -\set TablePart02 lineitem_partition_task_part_02 -SELECT usesysid AS userid FROM pg_user WHERE usename = current_user \gset -\set File_Basedir base/pgsql_job_cache -\set Table_File_00 :File_Basedir/job_:JobId/task_:PartitionTaskId/p_00000.:userid -\set Table_File_01 :File_Basedir/job_:JobId/task_:PartitionTaskId/p_00001.:userid -\set Table_File_02 :File_Basedir/job_:JobId/task_:PartitionTaskId/p_00002.:userid --- We assign a partition task and wait for it to complete. Note that we hardcode --- the partition function call string, including the job and task identifiers, --- into the argument in the task assignment function. This hardcoding is --- necessary as the current psql version does not perform variable interpolation --- for names inside single quotes. -SELECT task_tracker_assign_task(:JobId, :PartitionTaskId, - 'SELECT worker_range_partition_table(' - '401010, 801106, ''SELECT * FROM lineitem'', ' - '''l_orderkey'', 20, ARRAY[1000, 3000]::_int8)'); - task_tracker_assign_task ---------------------------------------------------------------------- - -(1 row) - -SELECT pg_sleep(4.0); - pg_sleep ---------------------------------------------------------------------- - -(1 row) - -SELECT task_tracker_task_status(:JobId, :PartitionTaskId); - task_tracker_task_status ---------------------------------------------------------------------- - 6 -(1 row) - -COPY :TablePart00 FROM :'Table_File_00'; -COPY :TablePart01 FROM :'Table_File_01'; -COPY :TablePart02 FROM :'Table_File_02'; -SELECT COUNT(*) FROM :TablePart00; - count ---------------------------------------------------------------------- - 1004 -(1 row) - -SELECT COUNT(*) FROM :TablePart02; - count ---------------------------------------------------------------------- - 8970 -(1 row) - --- We first compute the difference of partition tables against the base table. --- Then, we compute the difference of the base table against partitioned tables. -SELECT COUNT(*) AS diff_lhs_00 FROM ( - :SelectAll FROM :TablePart00 EXCEPT ALL - :SelectAll FROM lineitem WHERE :PartitionColumn < 1000 ) diff; - diff_lhs_00 ---------------------------------------------------------------------- - 0 -(1 row) - -SELECT COUNT(*) AS diff_lhs_01 FROM ( - :SelectAll FROM :TablePart01 EXCEPT ALL - :SelectAll FROM lineitem WHERE :PartitionColumn >= 1000 AND - :PartitionColumn < 3000 ) diff; - diff_lhs_01 ---------------------------------------------------------------------- - 0 -(1 row) - -SELECT COUNT(*) AS diff_lhs_02 FROM ( - :SelectAll FROM :TablePart02 EXCEPT ALL - :SelectAll FROM lineitem WHERE :PartitionColumn >= 3000 ) diff; - diff_lhs_02 ---------------------------------------------------------------------- - 0 -(1 row) - -SELECT COUNT(*) AS diff_rhs_00 FROM ( - :SelectAll FROM lineitem WHERE :PartitionColumn < 1000 EXCEPT ALL - :SelectAll FROM :TablePart00 ) diff; - diff_rhs_00 ---------------------------------------------------------------------- - 0 -(1 row) - -SELECT COUNT(*) AS diff_rhs_01 FROM ( - :SelectAll FROM lineitem WHERE :PartitionColumn >= 1000 AND - :PartitionColumn < 3000 EXCEPT ALL - :SelectAll FROM :TablePart01 ) diff; - diff_rhs_01 ---------------------------------------------------------------------- - 0 -(1 row) - -SELECT COUNT(*) AS diff_rhs_02 FROM ( - :SelectAll FROM lineitem WHERE :PartitionColumn >= 3000 EXCEPT ALL - :SelectAll FROM :TablePart02 ) diff; - diff_rhs_02 ---------------------------------------------------------------------- - 0 -(1 row) - diff --git a/src/test/regress/expected/with_executors.out b/src/test/regress/expected/with_executors.out index b7ac8f0fd..358fb7fb8 100644 --- a/src/test/regress/expected/with_executors.out +++ b/src/test/regress/expected/with_executors.out @@ -416,7 +416,11 @@ FROM users_table, cte_merge WHERE users_table.user_id = cte_merge.u_id; -ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker' + count +--------------------------------------------------------------------- + 4365606 +(1 row) + DROP SCHEMA with_executors CASCADE; NOTICE: drop cascades to 2 other objects DETAIL: drop cascades to table local_table diff --git a/src/test/regress/expected/worker_check_invalid_arguments.out b/src/test/regress/expected/worker_check_invalid_arguments.out deleted file mode 100644 index 3fa2d2335..000000000 --- a/src/test/regress/expected/worker_check_invalid_arguments.out +++ /dev/null @@ -1,100 +0,0 @@ --- --- WORKER_CHECK_INVALID_ARGUMENTS --- -SET citus.next_shard_id TO 1100000; -\set JobId 201010 -\set TaskId 101108 -\set Table_Name simple_binary_data_table -\set Partition_Column_Name '\'textcolumn\'' -\set Partition_Column_Type 25 -\set Partition_Count 2 -\set Select_Query_Text '\'SELECT * FROM simple_binary_data_table\'' -\set Bad_Partition_Column_Name '\'badcolumnname\'' -\set Bad_Partition_Column_Type 20 -\set Bad_Select_Query_Text '\'SELECT * FROM bad_table_name\'' --- Create simple table and insert a few rows into this table --- N.B. - These rows will be partitioned to files on disk then read back in the --- order the files are listed by a call to readdir; because this order is not --- predictable, the second column of these rows always has the same value, to --- avoid an error message differing based on file read order. -CREATE TABLE :Table_Name(textcolumn text, binarycolumn bytea); -COPY :Table_Name FROM stdin; -SELECT COUNT(*) FROM :Table_Name; - count ---------------------------------------------------------------------- - 2 -(1 row) - --- Check that we fail with bad SQL query -SELECT worker_range_partition_table(:JobId, :TaskId, :Bad_Select_Query_Text, - :Partition_Column_Name, :Partition_Column_Type, - ARRAY['aaa', 'some']::_text); -ERROR: relation "bad_table_name" does not exist -QUERY: SELECT * FROM bad_table_name --- Check that we fail with bad partition column name -SELECT worker_range_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Bad_Partition_Column_Name, :Partition_Column_Type, - ARRAY['aaa', 'some']::_text); -ERROR: could not find column name "badcolumnname" --- Check that we fail when partition column and split point types do not match -SELECT worker_range_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Partition_Column_Name, :Bad_Partition_Column_Type, - ARRAY['aaa', 'some']::_text); -ERROR: partition column type 20 and split point type 25 do not match --- Check that we fail with bad partition column type on hash partitioning -SELECT worker_hash_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Partition_Column_Name, :Bad_Partition_Column_Type, - ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]); -ERROR: partition column types 25 and 20 do not match --- Now, partition table data using valid arguments -SELECT worker_range_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Partition_Column_Name, :Partition_Column_Type, - ARRAY['aaa', 'some']::_text); - worker_range_partition_table ---------------------------------------------------------------------- - -(1 row) - --- Check that we fail to merge when the number of column names and column types --- do not match -SELECT worker_merge_files_into_table(:JobId, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea', 'integer']); -ERROR: column name array size: 2 and type array size: 3 do not match --- Check that we fail to merge when column types do not match underlying data -SELECT worker_merge_files_into_table(:JobId, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'integer']); -ERROR: invalid input syntax for integer: "\x0b50" -CONTEXT: COPY task_101108, line 1, column binarycolumn: "\x0b50" --- Check that we fail to merge when ids are wrong -SELECT worker_merge_files_into_table(-1, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea']); -ERROR: could not open directory "base/pgsql_job_cache/job_18446744073709551615/task_101108": No such file or directory -SELECT worker_merge_files_into_table(:JobId, -1, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea']); -ERROR: could not open directory "base/pgsql_job_cache/job_201010/task_4294967295": No such file or directory -SELECT worker_merge_files_and_run_query(-1, :TaskId, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); -ERROR: could not open directory "base/pgsql_job_cache/job_18446744073709551615/task_101108": No such file or directory -SELECT worker_merge_files_and_run_query(:JobId, -1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); -ERROR: could not open directory "base/pgsql_job_cache/job_201010/task_4294967295": No such file or directory --- Finally, merge partitioned files using valid arguments -SELECT worker_merge_files_into_table(:JobId, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea']); - worker_merge_files_into_table ---------------------------------------------------------------------- - -(1 row) - --- worker_execute_sql_task should only accept queries -select worker_execute_sql_task(0,0,'create table foo(a serial)',false); -ERROR: cannot execute utility commands diff --git a/src/test/regress/expected/worker_remove_files.out b/src/test/regress/expected/worker_remove_files.out index c53015952..e69de29bb 100644 --- a/src/test/regress/expected/worker_remove_files.out +++ b/src/test/regress/expected/worker_remove_files.out @@ -1,8 +0,0 @@ --- Clear job directory used by previous tests -\set JobId 201010 -SELECT task_tracker_cleanup_job(:JobId); - task_tracker_cleanup_job ---------------------------------------------------------------------- - -(1 row) - diff --git a/src/test/regress/input/multi_complex_count_distinct.source b/src/test/regress/input/multi_complex_count_distinct.source index 221053ab0..a30cb119c 100644 --- a/src/test/regress/input/multi_complex_count_distinct.source +++ b/src/test/regress/input/multi_complex_count_distinct.source @@ -34,7 +34,6 @@ SELECT create_distributed_table('lineitem_hash', 'l_orderkey', 'hash'); ANALYZE lineitem_hash; -SET citus.task_executor_type to "task-tracker"; -- count(distinct) is supported on top level query if there -- is a grouping on the partition key @@ -545,7 +544,6 @@ SELECT * ORDER BY 2 DESC, 1 LIMIT 10; -RESET citus.task_executor_type; -- count distinct pushdown is enabled SELECT * diff --git a/src/test/regress/multi_follower_schedule b/src/test/regress/multi_follower_schedule index d96386702..17b0fbd94 100644 --- a/src/test/regress/multi_follower_schedule +++ b/src/test/regress/multi_follower_schedule @@ -3,7 +3,6 @@ test: follower_single_node test: multi_follower_select_statements test: multi_follower_dml test: multi_follower_configure_followers -test: multi_follower_task_tracker # test that no tests leaked intermediate results. This should always be last test: ensure_no_intermediate_data_leak diff --git a/src/test/regress/multi_mx_schedule b/src/test/regress/multi_mx_schedule index 3767c302b..cc674cf7f 100644 --- a/src/test/regress/multi_mx_schedule +++ b/src/test/regress/multi_mx_schedule @@ -34,12 +34,13 @@ test: multi_mx_tpch_query7_nested multi_mx_ddl test: ch_bench_having_mx test: recursive_dml_queries_mx multi_mx_truncate_from_worker test: multi_mx_repartition_udt_prepare mx_foreign_key_to_reference_table -test: multi_mx_repartition_join_w1 multi_mx_repartition_join_w2 multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2 +test: multi_mx_repartition_join_w1 multi_mx_repartition_join_w2 test: multi_mx_metadata test: coordinator_evaluation coordinator_evaluation_modify coordinator_evaluation_select test: multi_mx_call test: multi_mx_function_call_delegation test: multi_mx_modifications local_shard_execution +test: multi_mx_repartition_udt_w1 multi_mx_repartition_udt_w2 test: local_shard_copy test: multi_mx_transaction_recovery test: multi_mx_modifying_xacts diff --git a/src/test/regress/multi_task_tracker_extra_schedule b/src/test/regress/multi_task_tracker_extra_schedule deleted file mode 100644 index e5f155cc1..000000000 --- a/src/test/regress/multi_task_tracker_extra_schedule +++ /dev/null @@ -1,123 +0,0 @@ -# ---------- -# $Id$ -# -# Regression tests for task tracker executor. This schedule runs tests -# in task tracker executor. Any test that do not explicitly set the task executor -# are expected to be placed here in addition to multi_schedule. -# -# Note that we use variant comparison files to test version dependent regression -# test results. For more information: -# http://www.postgresql.org/docs/current/static/regress-variant.html -# ---------- - -# --- -# Tests around schema changes, these are run first, so there's no preexisting objects. -# --- -test: multi_extension -test: multi_cluster_management -test: multi_table_ddl -test: multi_test_helpers multi_test_helpers_superuser -test: multi_test_catalog_views - -# ---------- -# The following distributed tests depend on creating a partitioned table and -# uploading data to it. -# ---------- -test: multi_create_table -test: multi_create_table_superuser -test: multi_master_protocol -test: multi_load_data multi_load_data_superuser - -# ---------- -# Miscellaneous tests to check our query planning behavior -# ---------- -test: multi_basic_queries multi_complex_expressions -test: multi_agg_distinct multi_limit_clause_approximate -test: multi_average_expression multi_working_columns -test: multi_array_agg multi_limit_clause -test: multi_agg_type_conversion multi_count_type_conversion -test: multi_hash_pruning -test: multi_query_directory_cleanup -test: multi_utility_statements -test: multi_dropped_column_aliases - -# ---------- -# Parallel TPC-H tests to check our distributed execution behavior -# ---------- -test: multi_tpch_query1 multi_tpch_query3 multi_tpch_query6 multi_tpch_query10 -test: multi_tpch_query12 multi_tpch_query14 multi_tpch_query19 -test: multi_tpch_query7 multi_tpch_query7_nested - -# ---------- -# Parallel tests to check our join order planning logic. Note that we load data -# below; and therefore these tests should come after the execution tests. -# ---------- -test: multi_load_more_data -test: multi_join_order_tpch_repartition - -# ---------- -# Tests to check our large record loading and shard deletion behavior -# ---------- -test: multi_load_large_records -test: multi_master_delete_protocol -test: multi_shard_modify - -# ---------- -# multi_create_schema tests creation, loading, and querying of a table in a new -# schema (namespace). -# ---------- -test: multi_create_schema - -# --------- -# multi_outer_join loads data to create shards to test outer join mappings -# --------- -test: multi_outer_join - -# --- -# Tests covering mostly modification queries and required preliminary -# functionality related to metadata, shard creation, shard pruning and -# "hacky" copy script for hash partitioned tables. -# Note that the order of the following tests are important. -# --- -test: multi_create_fdw -test: multi_distribution_metadata -test: multi_generate_ddl_commands -test: multi_create_shards -test: multi_prune_shard_list -test: multi_repair_shards -test: multi_modifications -test: multi_upsert -test: multi_simple_queries -test: multi_data_types -test: multi_utilities - -# --------- -# multi_copy creates hash and range-partitioned tables and performs COPY -# --------- -test: multi_copy - -# ---------- -# multi_large_shardid loads more lineitem data using high shard identifiers -# ---------- -test: multi_large_shardid - -# ---------- -# multi_drop_extension makes sure we can safely drop and recreate the extension -# ---------- -test: multi_drop_extension - -# ---------- -# multi_schema_support makes sure we can work with tables in schemas other than public with no problem -# ---------- -test: multi_schema_support - -# ---------- -# test that no tests leaked intermediate results. This should always be last -# ---------- -test: ensure_no_intermediate_data_leak - -# --------- -# ensures that we never leak any connection counts -# in the shared memory -# -------- -test: ensure_no_shared_connection_leak diff --git a/src/test/regress/output/multi_complex_count_distinct.source b/src/test/regress/output/multi_complex_count_distinct.source index 3363d080d..6a3f0a098 100644 --- a/src/test/regress/output/multi_complex_count_distinct.source +++ b/src/test/regress/output/multi_complex_count_distinct.source @@ -24,15 +24,14 @@ CREATE TABLE lineitem_hash ( l_comment varchar(44) not null, PRIMARY KEY(l_orderkey, l_linenumber) ); SELECT create_distributed_table('lineitem_hash', 'l_orderkey', 'hash'); - create_distributed_table --------------------------- - + create_distributed_table +--------------------------------------------------------------------- + (1 row) \copy lineitem_hash FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|' \copy lineitem_hash FROM '@abs_srcdir@/data/lineitem.2.data' with delimiter '|' ANALYZE lineitem_hash; -SET citus.task_executor_type to "task-tracker"; -- count(distinct) is supported on top level query if there -- is a grouping on the partition key SELECT @@ -41,8 +40,8 @@ SELECT GROUP BY l_orderkey ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- 14885 | 7 14884 | 7 14821 | 7 @@ -62,20 +61,20 @@ SELECT GROUP BY l_orderkey ORDER BY 2 DESC, 1 DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Limit Output: remote_scan.l_orderkey, remote_scan.count -> Sort Output: remote_scan.l_orderkey, remote_scan.count Sort Key: remote_scan.count DESC, remote_scan.l_orderkey DESC - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.l_orderkey, remote_scan.count Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_orderkey, count(DISTINCT l_partkey) AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_orderkey ORDER BY (count(DISTINCT l_partkey)) DESC, l_orderkey DESC LIMIT '10'::bigint - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> Limit Output: l_orderkey, (count(DISTINCT l_partkey)) -> Sort @@ -94,8 +93,8 @@ SELECT FROM lineitem_hash ORDER BY 1 DESC LIMIT 10; - count -------- + count +--------------------------------------------------------------------- 11661 (1 row) @@ -105,8 +104,8 @@ SELECT FROM lineitem_hash ORDER BY 1 DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Limit Output: (count(DISTINCT remote_scan.count)) -> Sort @@ -114,13 +113,13 @@ SELECT Sort Key: (count(DISTINCT remote_scan.count)) DESC -> Aggregate Output: count(DISTINCT remote_scan.count) - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.count Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_partkey AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_partkey - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> HashAggregate Output: l_partkey Group Key: lineitem_hash.l_partkey @@ -134,8 +133,8 @@ SELECT GROUP BY l_shipmode ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_shipmode | count -------------+------- + l_shipmode | count +--------------------------------------------------------------------- TRUCK | 1757 MAIL | 1730 AIR | 1702 @@ -152,8 +151,8 @@ SELECT GROUP BY l_shipmode ORDER BY 2 DESC, 1 DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Limit Output: remote_scan.l_shipmode, (count(DISTINCT remote_scan.count)) -> Sort @@ -165,13 +164,13 @@ SELECT -> Sort Output: remote_scan.l_shipmode, remote_scan.count Sort Key: remote_scan.l_shipmode DESC - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.l_shipmode, remote_scan.count Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_shipmode, l_partkey AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_shipmode, l_partkey - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> HashAggregate Output: l_shipmode, l_partkey Group Key: lineitem_hash.l_shipmode, lineitem_hash.l_partkey @@ -186,8 +185,8 @@ SELECT GROUP BY l_orderkey ORDER BY 3 DESC, 2 DESC, 1 LIMIT 10; - l_orderkey | count | count -------------+-------+------- + l_orderkey | count | count +--------------------------------------------------------------------- 226 | 7 | 7 1316 | 7 | 7 1477 | 7 | 7 @@ -207,20 +206,20 @@ SELECT GROUP BY l_orderkey ORDER BY 3 DESC, 2 DESC, 1 LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Limit Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 -> Sort Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 Sort Key: remote_scan.count_1 DESC, remote_scan.count DESC, remote_scan.l_orderkey - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_orderkey, count(DISTINCT l_partkey) AS count, count(DISTINCT l_shipmode) AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_orderkey ORDER BY (count(DISTINCT l_shipmode)) DESC, (count(DISTINCT l_partkey)) DESC, l_orderkey LIMIT '10'::bigint - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> Limit Output: l_orderkey, (count(DISTINCT l_partkey)), (count(DISTINCT l_shipmode)) -> Sort @@ -237,8 +236,8 @@ SELECT SELECT count(distinct l_orderkey), count(distinct l_partkey), count(distinct l_shipmode) FROM lineitem_hash; - count | count | count --------+-------+------- + count | count | count +--------------------------------------------------------------------- 2985 | 11661 | 7 (1 row) @@ -246,17 +245,17 @@ EXPLAIN (COSTS false, VERBOSE true) SELECT count(distinct l_orderkey), count(distinct l_partkey), count(distinct l_shipmode) FROM lineitem_hash; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Aggregate Output: count(DISTINCT remote_scan.count), count(DISTINCT remote_scan.count_1), count(DISTINCT remote_scan.count_2) - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.count, remote_scan.count_1, remote_scan.count_2 Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_orderkey AS count, l_partkey AS count, l_shipmode AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_orderkey, l_partkey, l_shipmode - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> HashAggregate Output: l_orderkey, l_partkey, l_shipmode Group Key: lineitem_hash.l_orderkey, lineitem_hash.l_partkey, lineitem_hash.l_shipmode @@ -270,8 +269,8 @@ SELECT count(distinct l_partkey), count(l_partkey), count(distinct l_shipmode), count(l_shipmode) FROM lineitem_hash; - count | count | count | count | count | count --------+-------+-------+-------+-------+------- + count | count | count | count | count | count +--------------------------------------------------------------------- 2985 | 12000 | 11661 | 12000 | 7 | 12000 (1 row) @@ -281,8 +280,8 @@ SELECT FROM lineitem_hash GROUP BY l_shipmode ORDER BY 1, 2 DESC, 3 DESC; - l_shipmode | count | count -------------+-------+------- + l_shipmode | count | count +--------------------------------------------------------------------- AIR | 1702 | 1327 FOB | 1700 | 1276 MAIL | 1730 | 1299 @@ -300,8 +299,8 @@ SELECT GROUP BY l_shipmode HAVING count(distinct l_orderkey) > 1300 ORDER BY 1, 2 DESC; - l_shipmode | count | count -------------+-------+------- + l_shipmode | count | count +--------------------------------------------------------------------- AIR | 1702 | 1327 TRUCK | 1757 | 1333 (2 rows) @@ -313,8 +312,8 @@ SELECT GROUP BY l_shipmode HAVING count(distinct l_orderkey) > 1300 ORDER BY 1, 2 DESC; - l_shipmode | count -------------+------- + l_shipmode | count +--------------------------------------------------------------------- AIR | 1702 TRUCK | 1757 (2 rows) @@ -327,8 +326,8 @@ SELECT GROUP BY l_shipmode HAVING count(distinct l_suppkey) > 1550 ORDER BY 1, 2 DESC; - l_shipmode | count | count -------------+-------+------- + l_shipmode | count | count +--------------------------------------------------------------------- AIR | 1702 | 1564 FOB | 1700 | 1571 MAIL | 1730 | 1573 @@ -345,8 +344,8 @@ SELECT GROUP BY l_shipmode HAVING count(distinct l_suppkey) > 1550 ORDER BY 1, 2 DESC; - l_shipmode | count -------------+------- + l_shipmode | count +--------------------------------------------------------------------- AIR | 1702 FOB | 1700 MAIL | 1730 @@ -363,8 +362,8 @@ SELECT GROUP BY l_shipmode HAVING count(distinct l_suppkey) > 1550 ORDER BY 1, 2 DESC; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Sort Output: remote_scan.l_shipmode, (count(DISTINCT remote_scan.count)) Sort Key: remote_scan.l_shipmode, (count(DISTINCT remote_scan.count)) DESC @@ -375,13 +374,13 @@ SELECT -> Sort Output: remote_scan.l_shipmode, remote_scan.count, remote_scan.worker_column_3 Sort Key: remote_scan.l_shipmode - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.l_shipmode, remote_scan.count, remote_scan.worker_column_3 Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_shipmode, l_partkey AS count, l_suppkey AS worker_column_3 FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_shipmode, l_partkey, l_suppkey - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> HashAggregate Output: l_shipmode, l_partkey, l_suppkey Group Key: lineitem_hash.l_shipmode, lineitem_hash.l_partkey, lineitem_hash.l_suppkey @@ -398,8 +397,8 @@ SELECT * GROUP BY l_orderkey) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- 14885 | 7 14884 | 7 14821 | 7 @@ -420,8 +419,8 @@ SELECT * GROUP BY l_partkey) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_partkey | count ------------+------- + l_partkey | count +--------------------------------------------------------------------- 199146 | 3 188804 | 3 177771 | 3 @@ -443,21 +442,43 @@ SELECT * GROUP BY l_partkey) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; - QUERY PLAN -------------------------------------------------------------------------- - Limit + QUERY PLAN +--------------------------------------------------------------------- + Custom Scan (Citus Adaptive) Output: remote_scan.l_partkey, remote_scan.count - -> Sort - Output: remote_scan.l_partkey, remote_scan.count - Sort Key: remote_scan.count DESC, remote_scan.l_partkey DESC - -> Custom Scan (Citus Task-Tracker) - Output: remote_scan.l_partkey, remote_scan.count - Task Count: 4 - Tasks Shown: None, not supported for re-partition queries - -> MapMergeJob - Map Task Count: 8 - Merge Task Count: 4 -(12 rows) + -> Distributed Subplan XXX_1 + -> HashAggregate + Output: remote_scan.l_partkey, COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) + Group Key: remote_scan.l_partkey + -> Custom Scan (Citus Adaptive) + Output: remote_scan.l_partkey, remote_scan.count + Task Count: 8 + Tasks Shown: One of 8 + -> Task + Query: SELECT l_partkey, count(DISTINCT l_orderkey) AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_partkey + Node: host=localhost port=xxxxx dbname=regression + -> GroupAggregate + Output: l_partkey, count(DISTINCT l_orderkey) + Group Key: lineitem_hash.l_partkey + -> Sort + Output: l_partkey, l_orderkey + Sort Key: lineitem_hash.l_partkey + -> Seq Scan on public.lineitem_hash_240000 lineitem_hash + Output: l_partkey, l_orderkey + Task Count: 1 + Tasks Shown: All + -> Task + Query: SELECT l_partkey, count FROM (SELECT intermediate_result.l_partkey, intermediate_result.count FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(l_partkey integer, count bigint)) sub ORDER BY count DESC, l_partkey DESC LIMIT 10 + Node: host=localhost port=xxxxx dbname=regression + -> Limit + Output: intermediate_result.l_partkey, intermediate_result.count + -> Sort + Output: intermediate_result.l_partkey, intermediate_result.count + Sort Key: intermediate_result.count DESC, intermediate_result.l_partkey DESC + -> Function Scan on pg_catalog.read_intermediate_result intermediate_result + Output: intermediate_result.l_partkey, intermediate_result.count + Function Call: read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) +(34 rows) -- count distinct with filters SELECT @@ -468,8 +489,8 @@ SELECT GROUP BY l_orderkey ORDER BY 2 DESC, 3 DESC, 1 LIMIT 10; - l_orderkey | count | count -------------+-------+------- + l_orderkey | count | count +--------------------------------------------------------------------- 4964 | 4 | 7 12005 | 4 | 7 5409 | 4 | 6 @@ -491,20 +512,20 @@ SELECT GROUP BY l_orderkey ORDER BY 2 DESC, 3 DESC, 1 LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Limit Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 -> Sort Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 Sort Key: remote_scan.count DESC, remote_scan.count_1 DESC, remote_scan.l_orderkey - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.l_orderkey, remote_scan.count, remote_scan.count_1 Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_orderkey, count(DISTINCT l_suppkey) FILTER (WHERE (l_shipmode OPERATOR(pg_catalog.=) 'AIR'::bpchar)) AS count, count(DISTINCT l_suppkey) AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_orderkey ORDER BY (count(DISTINCT l_suppkey) FILTER (WHERE (l_shipmode OPERATOR(pg_catalog.=) 'AIR'::bpchar))) DESC, (count(DISTINCT l_suppkey)) DESC, l_orderkey LIMIT '10'::bigint - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> Limit Output: l_orderkey, (count(DISTINCT l_suppkey) FILTER (WHERE (l_shipmode = 'AIR'::bpchar))), (count(DISTINCT l_suppkey)) -> Sort @@ -524,8 +545,8 @@ SELECT GROUP BY l_suppkey ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_suppkey | count ------------+------- + l_suppkey | count +--------------------------------------------------------------------- 7680 | 4 7703 | 3 7542 | 3 @@ -546,8 +567,8 @@ SELECT GROUP BY l_suppkey ORDER BY 2 DESC, 1 DESC LIMIT 10; - QUERY PLAN ----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- + QUERY PLAN +--------------------------------------------------------------------- Limit Output: remote_scan.l_suppkey, (count(DISTINCT remote_scan.count) FILTER (WHERE (remote_scan.count_1 = 'AIR'::bpchar))) -> Sort @@ -559,13 +580,13 @@ SELECT -> Sort Output: remote_scan.l_suppkey, remote_scan.count, remote_scan.count_1 Sort Key: remote_scan.l_suppkey DESC - -> Custom Scan (Citus Task-Tracker) + -> Custom Scan (Citus Adaptive) Output: remote_scan.l_suppkey, remote_scan.count, remote_scan.count_1 Task Count: 8 Tasks Shown: One of 8 -> Task Query: SELECT l_suppkey, l_partkey AS count, l_shipmode AS count FROM lineitem_hash_240000 lineitem_hash WHERE true GROUP BY l_suppkey, l_partkey, l_shipmode - Node: host=localhost port=57637 dbname=regression + Node: host=localhost port=xxxxx dbname=regression -> HashAggregate Output: l_suppkey, l_partkey, l_shipmode Group Key: lineitem_hash.l_suppkey, lineitem_hash.l_partkey, lineitem_hash.l_shipmode @@ -577,8 +598,8 @@ SELECT SELECT count(DISTINCT l_orderkey) FILTER (WHERE l_shipmode = 'AIR') FROM lineitem_hash; - count -------- + count +--------------------------------------------------------------------- 1327 (1 row) @@ -586,8 +607,8 @@ SELECT SELECT count(DISTINCT l_partkey) FILTER (WHERE l_shipmode = 'AIR') FROM lineitem_hash; - count -------- + count +--------------------------------------------------------------------- 1702 (1 row) @@ -596,8 +617,8 @@ SELECT count(DISTINCT l_partkey), count(DISTINCT l_shipdate) FROM lineitem_hash; - count | count | count --------+-------+------- + count | count | count +--------------------------------------------------------------------- 1702 | 11661 | 2470 (1 row) @@ -610,8 +631,8 @@ SELECT * GROUP BY l_orderkey) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- 14885 | 7 14884 | 7 14821 | 7 @@ -633,8 +654,8 @@ SELECT * GROUP BY l_orderkey) sub ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- 12005 | 4 5409 | 4 4964 | 4 @@ -658,8 +679,8 @@ SELECT * WHERE count > 0 ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- 12005 | 4 5409 | 4 4964 | 4 @@ -682,8 +703,8 @@ SELECT * WHERE count > 0 ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- 14275 | 7 14181 | 7 13605 | 7 @@ -729,8 +750,8 @@ SELECT * GROUP BY l_orderkey) sub ORDER BY 2 DESC, 1 DESC LIMIT 0; - l_orderkey | count -------------+------- + l_orderkey | count +--------------------------------------------------------------------- (0 rows) -- multiple nested subquery @@ -763,8 +784,8 @@ SELECT total ORDER BY total_avg_count DESC; - total | total_avg_count --------+-------------------- + total | total_avg_count +--------------------------------------------------------------------- 1 | 3.6000000000000000 6 | 2.8333333333333333 10 | 2.6000000000000000 @@ -795,8 +816,8 @@ SELECT * ORDER BY 1 DESC, 2 DESC LIMIT 10; - count | l_shipdate --------+------------ + count | l_shipdate +--------------------------------------------------------------------- 14 | 07-30-1997 13 | 05-26-1998 13 | 08-08-1997 @@ -823,8 +844,8 @@ SELECT * ORDER BY 2 DESC, 1 DESC LIMIT 10; - l_quantity | count -------------+------- + l_quantity | count +--------------------------------------------------------------------- 48.00 | 13 47.00 | 13 37.00 | 13 @@ -855,15 +876,15 @@ SELECT * ORDER BY 1 DESC, 2 DESC LIMIT 10; - avg | l_shipmode --------------------------+------------ - 44.82904609027336300064 | MAIL - 44.80704536679536679537 | SHIP - 44.68891732736572890026 | AIR - 44.34106724470134874759 | REG AIR - 43.12739987269255251432 | FOB - 43.07299253636938646426 | RAIL - 40.50298377916903813318 | TRUCK + avg | l_shipmode +--------------------------------------------------------------------- + 44.82904609027336300064 | MAIL + 44.80704536679536679537 | SHIP + 44.68891732736572890026 | AIR + 44.34106724470134874759 | REG AIR + 43.12739987269255251432 | FOB + 43.07299253636938646426 | RAIL + 40.50298377916903813318 | TRUCK (7 rows) -- count DISTINCT CASE WHEN expression @@ -881,8 +902,8 @@ SELECT * GROUP BY l_shipdate) sub ORDER BY 1 DESC LIMIT 10; - avg ------ + avg +--------------------------------------------------------------------- 7 6 6 @@ -908,8 +929,8 @@ SELECT * ORDER BY 2 DESC,1 DESC LIMIT 10; - l_shipmode | count -------------+------- + l_shipmode | count +--------------------------------------------------------------------- TRUCK | 1689 MAIL | 1683 FOB | 1655 @@ -930,8 +951,8 @@ SELECT GROUP BY 1 ORDER BY 1 DESC LIMIT 5; - l_orderkey | count | count -------------+-------+------- + l_orderkey | count | count +--------------------------------------------------------------------- 14947 | 2 | 1 14946 | 2 | 1 14945 | 6 | 1 @@ -952,8 +973,8 @@ FROM GROUP BY 1 ORDER BY 1 DESC LIMIT 5; - user_id | count | count | count ----------+-------+-------+------- + user_id | count | count | count +--------------------------------------------------------------------- 6 | 11 | 1 | 1 5 | 27 | 1 | 1 4 | 24 | 1 | 1 @@ -968,9 +989,9 @@ CREATE TYPE test_item AS ); CREATE TABLE test_count_distinct_array (key int, value int , value_arr test_item[]); SELECT create_distributed_table('test_count_distinct_array', 'key'); - create_distributed_table --------------------------- - + create_distributed_table +--------------------------------------------------------------------- + (1 row) INSERT INTO test_count_distinct_array SELECT i, i, ARRAY[(i,i)::test_item] FROM generate_Series(0, 1000) i; @@ -986,8 +1007,8 @@ FROM GROUP BY 1 ORDER BY 1 DESC LIMIT 5; - key | count | count | count -------+-------+-------+------- + key | count | count | count +--------------------------------------------------------------------- 1000 | 1 | 1 | 1 999 | 1 | 1 | 1 998 | 1 | 1 | 1 @@ -1007,7 +1028,7 @@ SELECT * ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute aggregate (distinct) -DETAIL: Only count(distinct) aggregate is supported in subqueries +DETAIL: table partitioning is unsuitable for aggregate (distinct) SELECT * FROM ( SELECT @@ -1017,7 +1038,7 @@ SELECT * ORDER BY 2 DESC, 1 DESC LIMIT 10; ERROR: cannot compute aggregate (distinct) -DETAIL: Only count(distinct) aggregate is supported in subqueries +DETAIL: table partitioning is unsuitable for aggregate (distinct) -- whole row references, oid, and ctid are not supported in count distinct -- test table does not have oid or ctid enabled, so tests for them are skipped SELECT * @@ -1057,8 +1078,8 @@ SELECT * WHERE year = 1995 ORDER BY 2 DESC, 1 LIMIT 10; - l_shipdate | distinct_part | year -------------+---------------+------ + l_shipdate | distinct_part | year +--------------------------------------------------------------------- 11-29-1995 | 5 | 1995 03-24-1995 | 4 | 1995 09-18-1995 | 4 | 1995 @@ -1071,7 +1092,6 @@ SELECT * 10-30-1995 | 3 | 1995 (10 rows) -RESET citus.task_executor_type; -- count distinct pushdown is enabled SELECT * FROM ( @@ -1089,8 +1109,8 @@ SELECT * WHERE year = 1995 ORDER BY 2 DESC, 1 LIMIT 10; - l_shipdate | distinct_part | year -------------+---------------+------ + l_shipdate | distinct_part | year +--------------------------------------------------------------------- 11-29-1995 | 5 | 1995 03-24-1995 | 4 | 1995 09-18-1995 | 4 | 1995 @@ -1119,8 +1139,8 @@ SELECT * WHERE year = 1995 ORDER BY 2 DESC, 1 LIMIT 10; - l_shipdate | distinct_part | year -------------+---------------+------ + l_shipdate | distinct_part | year +--------------------------------------------------------------------- 11-29-1995 | 5 | 1995 03-24-1995 | 4 | 1995 09-18-1995 | 4 | 1995 diff --git a/src/test/regress/pg_regress_multi.pl b/src/test/regress/pg_regress_multi.pl index 9bdbb1c19..51148554b 100755 --- a/src/test/regress/pg_regress_multi.pl +++ b/src/test/regress/pg_regress_multi.pl @@ -410,6 +410,7 @@ push(@pgOptions, '-c', "wal_level=logical"); push(@pgOptions, '-c', "citus.shard_count=4"); push(@pgOptions, '-c', "citus.shard_max_size=1500kB"); push(@pgOptions, '-c', "citus.max_running_tasks_per_node=4"); +push(@pgOptions, '-c', "citus.repartition_join_bucket_count_per_node=2"); push(@pgOptions, '-c', "citus.expire_cached_shards=on"); push(@pgOptions, '-c', "citus.sort_returning=on"); push(@pgOptions, '-c', "citus.task_tracker_delay=10ms"); diff --git a/src/test/regress/spec/isolation_append_copy_vs_all.spec b/src/test/regress/spec/isolation_append_copy_vs_all.spec index 476ed2720..5ef2a616b 100644 --- a/src/test/regress/spec/isolation_append_copy_vs_all.spec +++ b/src/test/regress/spec/isolation_append_copy_vs_all.spec @@ -26,7 +26,7 @@ step "s1-router-select" { SELECT * FROM append_copy WHERE id = 1; } step "s1-real-time-select" { SELECT * FROM append_copy ORDER BY 1, 2; } step "s1-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM append_copy AS t1 JOIN append_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s1-insert" { INSERT INTO append_copy VALUES(0, 'k', 0); } @@ -59,7 +59,7 @@ step "s2-router-select" { SELECT * FROM append_copy WHERE id = 1; } step "s2-real-time-select" { SELECT * FROM append_copy ORDER BY 1, 2; } step "s2-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM append_copy AS t1 JOIN append_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s2-insert" { INSERT INTO append_copy VALUES(0, 'k', 0); } diff --git a/src/test/regress/spec/isolation_hash_copy_vs_all.spec b/src/test/regress/spec/isolation_hash_copy_vs_all.spec index 9b607c2a5..46571f0f7 100644 --- a/src/test/regress/spec/isolation_hash_copy_vs_all.spec +++ b/src/test/regress/spec/isolation_hash_copy_vs_all.spec @@ -26,7 +26,7 @@ step "s1-router-select" { SELECT * FROM hash_copy WHERE id = 1; } step "s1-real-time-select" { SELECT * FROM hash_copy ORDER BY 1, 2; } step "s1-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s1-insert" { INSERT INTO hash_copy VALUES(0, 'k', 0); } @@ -66,7 +66,7 @@ step "s2-router-select" { SELECT * FROM hash_copy WHERE id = 1; } step "s2-real-time-select" { SELECT * FROM hash_copy ORDER BY 1, 2; } step "s2-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM hash_copy AS t1 JOIN hash_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s2-insert" { INSERT INTO hash_copy VALUES(0, 'k', 0); } diff --git a/src/test/regress/spec/isolation_partitioned_copy_vs_all.spec b/src/test/regress/spec/isolation_partitioned_copy_vs_all.spec index 83b683845..be4dc8e70 100644 --- a/src/test/regress/spec/isolation_partitioned_copy_vs_all.spec +++ b/src/test/regress/spec/isolation_partitioned_copy_vs_all.spec @@ -29,7 +29,7 @@ step "s1-router-select" { SELECT * FROM partitioned_copy WHERE id = 1; } step "s1-real-time-select" { SELECT * FROM partitioned_copy ORDER BY 1, 2; } step "s1-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s1-insert" { INSERT INTO partitioned_copy VALUES(0, 'k', 0); } @@ -62,7 +62,7 @@ step "s2-router-select" { SELECT * FROM partitioned_copy WHERE id = 1; } step "s2-real-time-select" { SELECT * FROM partitioned_copy ORDER BY 1, 2; } step "s2-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM partitioned_copy AS t1 JOIN partitioned_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s2-insert" { INSERT INTO partitioned_copy VALUES(0, 'k', 0); } diff --git a/src/test/regress/spec/isolation_range_copy_vs_all.spec b/src/test/regress/spec/isolation_range_copy_vs_all.spec index 590553486..f4ad255c5 100644 --- a/src/test/regress/spec/isolation_range_copy_vs_all.spec +++ b/src/test/regress/spec/isolation_range_copy_vs_all.spec @@ -26,7 +26,7 @@ step "s1-router-select" { SELECT * FROM range_copy WHERE id = 1; } step "s1-real-time-select" { SELECT * FROM range_copy ORDER BY 1, 2; } step "s1-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM range_copy AS t1 JOIN range_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s1-insert" { INSERT INTO range_copy VALUES(0, 'k', 0); } @@ -61,7 +61,7 @@ step "s2-router-select" { SELECT * FROM range_copy WHERE id = 1; } step "s2-real-time-select" { SELECT * FROM range_copy ORDER BY 1, 2; } step "s2-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM range_copy AS t1 JOIN range_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s2-insert" { INSERT INTO range_copy VALUES(0, 'k', 0); } diff --git a/src/test/regress/spec/isolation_reference_copy_vs_all.spec b/src/test/regress/spec/isolation_reference_copy_vs_all.spec index b3e7d366e..8ccdd3c83 100644 --- a/src/test/regress/spec/isolation_reference_copy_vs_all.spec +++ b/src/test/regress/spec/isolation_reference_copy_vs_all.spec @@ -26,8 +26,7 @@ step "s1-router-select" { SELECT * FROM reference_copy WHERE id = 1; } step "s1-real-time-select" { SELECT * FROM reference_copy ORDER BY 1, 2; } step "s1-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s1-insert" { INSERT INTO reference_copy VALUES(0, 'k', 0); } step "s1-insert-select" { INSERT INTO reference_copy SELECT * FROM reference_copy; } @@ -59,8 +58,7 @@ step "s2-router-select" { SELECT * FROM reference_copy WHERE id = 1; } step "s2-real-time-select" { SELECT * FROM reference_copy ORDER BY 1, 2; } step "s2-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; - SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; + SELECT * FROM reference_copy AS t1 JOIN reference_copy AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s2-insert" { INSERT INTO reference_copy VALUES(0, 'k', 0); } step "s2-insert-select" { INSERT INTO reference_copy SELECT * FROM reference_copy; } diff --git a/src/test/regress/spec/isolation_select_vs_all.spec b/src/test/regress/spec/isolation_select_vs_all.spec index 20a66c634..9f9c01de7 100644 --- a/src/test/regress/spec/isolation_select_vs_all.spec +++ b/src/test/regress/spec/isolation_select_vs_all.spec @@ -33,7 +33,7 @@ step "s1-router-select" { SELECT * FROM select_append WHERE id = 1; } step "s1-real-time-select" { SELECT * FROM select_append ORDER BY 1, 2; } step "s1-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s1-insert" { INSERT INTO select_append VALUES(0, 'k', 0); } @@ -64,7 +64,7 @@ step "s2-router-select" { SELECT * FROM select_append WHERE id = 1; } step "s2-real-time-select" { SELECT * FROM select_append ORDER BY 1, 2; } step "s2-task-tracker-select" { - SET citus.task_executor_type TO "task-tracker"; + SET citus.enable_repartition_joins TO ON; SELECT * FROM select_append AS t1 JOIN select_append AS t2 ON t1.id = t2.int_data ORDER BY 1, 2, 3, 4; } step "s2-insert" { INSERT INTO select_append VALUES(0, 'k', 0); } diff --git a/src/test/regress/sql/adaptive_executor.sql b/src/test/regress/sql/adaptive_executor.sql index 1002fb8f4..a744437f1 100644 --- a/src/test/regress/sql/adaptive_executor.sql +++ b/src/test/regress/sql/adaptive_executor.sql @@ -13,7 +13,6 @@ INSERT INTO test VALUES (3,2); -- Set a very high slow start to avoid opening parallel connections SET citus.executor_slow_start_interval TO '60s'; SET citus.max_adaptive_executor_pool_size TO 2; -SET citus.task_executor_type TO 'adaptive'; BEGIN; SELECT count(*) FROM test a JOIN (SELECT x, pg_sleep(0.1) FROM test) b USING (x); diff --git a/src/test/regress/sql/adaptive_executor_repartition.sql b/src/test/regress/sql/adaptive_executor_repartition.sql index b184aee83..fb9c5a208 100644 --- a/src/test/regress/sql/adaptive_executor_repartition.sql +++ b/src/test/regress/sql/adaptive_executor_repartition.sql @@ -1,7 +1,6 @@ CREATE SCHEMA adaptive_executor; SET search_path TO adaptive_executor; -SET citus.task_executor_type to 'adaptive'; SET citus.shard_replication_factor to 1; SET citus.enable_repartition_joins TO true; diff --git a/src/test/regress/sql/aggregate_support.sql b/src/test/regress/sql/aggregate_support.sql index 2267a53b6..c70c8be35 100644 --- a/src/test/regress/sql/aggregate_support.sql +++ b/src/test/regress/sql/aggregate_support.sql @@ -180,7 +180,6 @@ select floor(val/2), corr(valf, valf + val) from aggdata group by floor(val/2) h select array_agg(val order by valf) from aggdata; -- Test TransformSubqueryNode -SET citus.task_executor_type to "task-tracker"; select * FROM ( SELECT key, mode() within group (order by floor(agg1.val/2)) m from aggdata agg1 @@ -190,7 +189,8 @@ select * FROM ( select * FROM ( SELECT key k, avg(distinct floor(agg1.val/2)) m from aggdata agg1 group by key -) subq; +) subq +order by k,m; -- Test TransformsSubqueryNode with group by not in FROM (failed in past) select count(*) FROM ( @@ -198,7 +198,6 @@ select count(*) FROM ( group by key ) subq; -RESET citus.task_executor_type; select key, count(distinct aggdata) from aggdata group by key order by 1, 2; diff --git a/src/test/regress/sql/limit_intermediate_size.sql b/src/test/regress/sql/limit_intermediate_size.sql index 6207faef1..adfabd1ae 100644 --- a/src/test/regress/sql/limit_intermediate_size.sql +++ b/src/test/regress/sql/limit_intermediate_size.sql @@ -1,6 +1,5 @@ SET citus.enable_repartition_joins to ON; -SET citus.task_executor_type to 'task-tracker'; -- prevent PG 11 - PG 12 outputs to diverge SET citus.enable_cte_inlining TO false; @@ -100,7 +99,7 @@ WITH cte AS ( SELECT * FROM cte2, cte3 WHERE cte2.user_id = cte3.user_id AND cte2.user_id = 1 AND EXISTS (select * from cte2, cte3) ) -SELECT * FROM cte WHERE EXISTS (select * from cte); +SELECT count(*) FROM cte WHERE EXISTS (select * from cte); SET citus.max_intermediate_result_size TO 3; @@ -115,7 +114,7 @@ WITH cte AS ( ) SELECT * FROM cte2, cte3 WHERE cte2.value_1 IN (SELECT value_2 FROM cte3) ) -SELECT * FROM cte; +SELECT count(*) FROM cte; -- this will fail in real_time_executor diff --git a/src/test/regress/sql/multi_binary_master_copy_format.sql b/src/test/regress/sql/multi_binary_master_copy_format.sql index 785ed3d82..5a0f9a073 100644 --- a/src/test/regress/sql/multi_binary_master_copy_format.sql +++ b/src/test/regress/sql/multi_binary_master_copy_format.sql @@ -9,12 +9,10 @@ SET citus.next_shard_id TO 430000; -- Try binary master copy for different executors SET citus.binary_master_copy_format TO 'on'; -SET citus.task_executor_type TO 'task-tracker'; SELECT count(*) FROM lineitem; SELECT l_shipmode FROM lineitem WHERE l_partkey = 67310 OR l_partkey = 155190; -RESET citus.task_executor_type; SELECT count(*) FROM lineitem; SELECT l_shipmode FROM lineitem WHERE l_partkey = 67310 OR l_partkey = 155190; diff --git a/src/test/regress/sql/multi_complex_expressions.sql b/src/test/regress/sql/multi_complex_expressions.sql index 0aa6cdb39..f6c49695d 100644 --- a/src/test/regress/sql/multi_complex_expressions.sql +++ b/src/test/regress/sql/multi_complex_expressions.sql @@ -167,7 +167,6 @@ SELECT * FROM (SELECT o_orderkey FROM orders ORDER BY o_orderkey OFFSET 20) sq O SELECT o_orderkey FROM orders ORDER BY o_orderkey LIMIT 10 OFFSET 20; -- LIMIT/OFFSET with a subquery -SET citus.task_executor_type TO 'task-tracker'; SELECT customer_keys.o_custkey, @@ -205,7 +204,6 @@ ORDER BY customer_keys.o_custkey DESC LIMIT 10 OFFSET 20; -RESET citus.task_executor_type; SET client_min_messages TO DEBUG1; -- Ensure that we push down LIMIT and OFFSET properly diff --git a/src/test/regress/sql/multi_cross_shard.sql b/src/test/regress/sql/multi_cross_shard.sql index 8af073e42..5e716c932 100644 --- a/src/test/regress/sql/multi_cross_shard.sql +++ b/src/test/regress/sql/multi_cross_shard.sql @@ -4,6 +4,8 @@ -- Tests to log cross shard queries according to error log level -- +SET citus.enable_repartition_joins to ON; + -- Create a distributed table and add data to it CREATE TABLE multi_task_table ( @@ -112,8 +114,7 @@ INSERT INTO tt2 VALUES(1, 'Ahmet', 5); INSERT INTO tt2 VALUES(2, 'Mehmet', 15); -- Should notice since it is a task-tracker query -SET citus.task_executor_type to "task-tracker"; -SELECT tt1.id, tt2.count from tt1,tt2 where tt1.id = tt2.id; +SELECT tt1.id, tt2.count from tt1,tt2 where tt1.id = tt2.id ORDER BY 1; SET citus.task_executor_type to DEFAULT; diff --git a/src/test/regress/sql/multi_explain.sql b/src/test/regress/sql/multi_explain.sql index de516bd99..191903646 100644 --- a/src/test/regress/sql/multi_explain.sql +++ b/src/test/regress/sql/multi_explain.sql @@ -6,8 +6,8 @@ SET citus.next_shard_id TO 570000; \a\t -RESET citus.task_executor_type; SET citus.explain_distributed_queries TO on; +SET citus.enable_repartition_joins to ON; -- Function that parses explain output as JSON CREATE FUNCTION explain_json(query text) @@ -447,7 +447,6 @@ EXPLAIN (COSTS FALSE) WHERE orders_hash_part.o_orderkey = lineitem_hash_part.l_orderkey; -- Test track tracker -SET citus.task_executor_type TO 'task-tracker'; EXPLAIN (COSTS FALSE) SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; @@ -522,7 +521,6 @@ PREPARE task_tracker_query AS SELECT avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030; EXPLAIN (COSTS FALSE) EXECUTE task_tracker_query; -RESET citus.task_executor_type; PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5; EXPLAIN EXECUTE router_executor_query; diff --git a/src/test/regress/sql/multi_follower_task_tracker.sql b/src/test/regress/sql/multi_follower_task_tracker.sql deleted file mode 100644 index 886be84d6..000000000 --- a/src/test/regress/sql/multi_follower_task_tracker.sql +++ /dev/null @@ -1,22 +0,0 @@ -\c - - - :master_port - --- do some setup - -CREATE TABLE tab(a int, b int); -SELECT create_distributed_table('tab', 'a'); - -INSERT INTO tab (a, b) VALUES (1, 1); -INSERT INTO tab (a, b) VALUES (1, 2); - -\c - - - :follower_master_port - -RESET citus.task_executor_type; -SELECT * FROM tab; -SET citus.task_executor_type TO 'task-tracker'; -SELECT * FROM tab; - --- clean up - -\c - - - :master_port - -DROP TABLE tab; diff --git a/src/test/regress/sql/multi_hash_pruning.sql b/src/test/regress/sql/multi_hash_pruning.sql index af881be0b..5f2115c17 100644 --- a/src/test/regress/sql/multi_hash_pruning.sql +++ b/src/test/regress/sql/multi_hash_pruning.sql @@ -294,7 +294,6 @@ SELECT count(*) FROM orders_hash_partitioned SELECT count(*) FROM orders_hash_partitioned WHERE NOT (o_orderkey = 2 AND o_orderkey = 3); -SET citus.task_executor_type TO 'adaptive'; -- Check that subquery NOT is pruned when ANDed to a valid constraint SELECT count(*) FROM orders_hash_partitioned diff --git a/src/test/regress/sql/multi_join_order_additional.sql b/src/test/regress/sql/multi_join_order_additional.sql index 869ae0889..1d30900dd 100644 --- a/src/test/regress/sql/multi_join_order_additional.sql +++ b/src/test/regress/sql/multi_join_order_additional.sql @@ -10,6 +10,7 @@ SET citus.next_shard_id TO 650000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise +SET citus.enable_repartition_joins to ON; SET citus.shard_count to 2; SET citus.shard_replication_factor to 1; RESET client_min_messages; diff --git a/src/test/regress/sql/multi_join_order_tpch_repartition.sql b/src/test/regress/sql/multi_join_order_tpch_repartition.sql index 8e9c59713..3060f8123 100644 --- a/src/test/regress/sql/multi_join_order_tpch_repartition.sql +++ b/src/test/regress/sql/multi_join_order_tpch_repartition.sql @@ -11,6 +11,7 @@ SET citus.next_shard_id TO 660000; SET citus.explain_distributed_queries TO off; SET citus.log_multi_join_order TO TRUE; SET citus.task_executor_type = 'task-tracker'; -- can't explain all queries otherwise +SET citus.enable_repartition_joins to ON; SET client_min_messages TO LOG; -- The following queries are basically the same as the ones in tpch_small diff --git a/src/test/regress/sql/multi_multiuser.sql b/src/test/regress/sql/multi_multiuser.sql index 10e19f8a4..3df4907f1 100644 --- a/src/test/regress/sql/multi_multiuser.sql +++ b/src/test/regress/sql/multi_multiuser.sql @@ -106,27 +106,22 @@ INSERT INTO test VALUES (2); SELECT count(*) FROM test; SELECT count(*) FROM test WHERE id = 1; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; SELECT count(*), min(current_user) FROM test; -- test re-partition query (needs to transmit intermediate results) SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -SET citus.task_executor_type TO 'adaptive'; SET citus.enable_repartition_joins TO true; SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); -RESET citus.task_executor_type; -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); --- create a task that other users should not be able to inspect -SELECT task_tracker_assign_task(1, 1, 'SELECT 1'); - -- check read permission SET ROLE read_access; @@ -142,31 +137,24 @@ INSERT INTO test VALUES (2); SELECT count(*) FROM test; SELECT count(*) FROM test WHERE id = 1; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; SELECT count(*), min(current_user) FROM test; -- test re-partition query (needs to transmit intermediate results) SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -SET citus.task_executor_type TO 'adaptive'; SET citus.enable_repartition_joins TO true; SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); --- should not be able to access tasks or jobs belonging to a different user -SELECT task_tracker_task_status(1, 1); -SELECT task_tracker_assign_task(1, 2, 'SELECT 1'); -SELECT task_tracker_cleanup_job(1); - -- should not be allowed to take aggressive locks on table BEGIN; SELECT lock_relation_if_exists('test', 'ACCESS SHARE'); SELECT lock_relation_if_exists('test', 'EXCLUSIVE'); ABORT; -RESET citus.task_executor_type; -- check no permission SET ROLE no_access; @@ -178,20 +166,18 @@ INSERT INTO test VALUES (2); SELECT count(*) FROM test; SELECT count(*) FROM test WHERE id = 1; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; SELECT count(*), min(current_user) FROM test; -- test re-partition query SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -SET citus.task_executor_type TO 'adaptive'; SET citus.enable_repartition_joins TO true; SELECT count(*) FROM test a JOIN test b ON (a.val = b.val) WHERE a.id = 1 AND b.id = 2; -- should not be able to transmit directly COPY "postgresql.conf" TO STDOUT WITH (format transmit); -RESET citus.task_executor_type; -- should be able to use intermediate results as any user BEGIN; @@ -227,8 +213,6 @@ RESET ROLE; SELECT create_distributed_table('my_table', 'id'); SELECT result FROM run_command_on_workers($$SELECT tableowner FROM pg_tables WHERE tablename LIKE 'my_table_%' LIMIT 1$$); -SELECT task_tracker_cleanup_job(1); - -- table should be distributable by super user when it has data in there SET ROLE full_access; CREATE TABLE my_table_with_data (id integer, val integer); @@ -407,13 +391,21 @@ RESET ROLE; -- to test access to files created during repartition we will create some on worker 1 \c - - - :worker_1_port +CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) + RETURNS void + AS 'citus' + LANGUAGE C STRICT; SET ROLE full_access; SELECT worker_hash_partition_table(42,1,'SELECT a FROM generate_series(1,100) AS a', 'a', 23, ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]); RESET ROLE; - -- all attempts for transfer are initiated from other workers \c - - - :worker_2_port + +CREATE OR REPLACE FUNCTION citus_rm_job_directory(bigint) + RETURNS void + AS 'citus' + LANGUAGE C STRICT; -- super user should not be able to copy files created by a user SELECT worker_fetch_partition_file(42, 1, 1, 1, 'localhost', :worker_1_port); @@ -433,13 +425,6 @@ SET ROLE usage_access; SELECT worker_merge_files_into_table(42, 1, ARRAY['a'], ARRAY['integer']); RESET ROLE; -SET ROLE full_access; --- use the side effect of this function to have a schema to use, otherwise only the super --- user could call worker_merge_files_into_table and store the results in public, which is --- not what we want -SELECT task_tracker_assign_task(42, 1, 'SELECT 1'); -RESET ROLE; - -- test that no other user can merge the downloaded file after the task is being tracked SET ROLE usage_access; SELECT worker_merge_files_into_table(42, 1, ARRAY['a'], ARRAY['integer']); @@ -457,51 +442,22 @@ SELECT count(*) FROM pg_merge_job_0042.task_000001; DROP TABLE pg_merge_job_0042.task_000001; -- drop table so we can reuse the same files for more tests RESET ROLE; --- test that no other user can merge files and run query on the already fetched files -SET ROLE usage_access; -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); -RESET ROLE; - --- test that the super user is unable to read the contents of the partitioned files after --- trying to merge with run query -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); SELECT count(*) FROM pg_merge_job_0042.task_000001_merge; SELECT count(*) FROM pg_merge_job_0042.task_000001; DROP TABLE pg_merge_job_0042.task_000001, pg_merge_job_0042.task_000001_merge; -- drop table so we can reuse the same files for more tests --- test that the owner of the task can merge files and run query correctly -SET ROLE full_access; -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); - --- test that owner of task cannot execute arbitrary sql -SELECT worker_merge_files_and_run_query(42, 1, - 'CREATE TABLE task_000002_merge(merge_column_0 int)', - 'DROP USER usage_access' -); - -SELECT worker_merge_files_and_run_query(42, 1, - 'DROP USER usage_access', - 'CREATE TABLE task_000002 (a) AS SELECT sum(merge_column_0) FROM task_000002_merge' -); - SELECT count(*) FROM pg_merge_job_0042.task_000001_merge; SELECT count(*) FROM pg_merge_job_0042.task_000001; DROP TABLE pg_merge_job_0042.task_000001, pg_merge_job_0042.task_000001_merge; -- drop table so we can reuse the same files for more tests RESET ROLE; +SELECT citus_rm_job_directory(42::bigint); + +\c - - - :worker_1_port +SELECT citus_rm_job_directory(42::bigint); + \c - - - :master_port -SELECT run_command_on_workers($$SELECT task_tracker_cleanup_job(42);$$); - DROP SCHEMA full_access_user_schema CASCADE; DROP TABLE my_table, diff --git a/src/test/regress/sql/multi_mx_explain.sql b/src/test/regress/sql/multi_mx_explain.sql index 47eb4433f..9417b94f3 100644 --- a/src/test/regress/sql/multi_mx_explain.sql +++ b/src/test/regress/sql/multi_mx_explain.sql @@ -9,7 +9,6 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 1320000; \a\t -RESET citus.task_executor_type; SET citus.explain_distributed_queries TO on; VACUUM ANALYZE lineitem_mx; @@ -166,7 +165,6 @@ SELECT true AS valid FROM explain_json($$ SELECT avg(l_linenumber) FROM lineitem_mx WHERE l_orderkey > 9030$$); -- Test track tracker -SET citus.task_executor_type TO 'task-tracker'; SET citus.explain_all_tasks TO off; EXPLAIN (COSTS FALSE) diff --git a/src/test/regress/sql/multi_mx_reference_table.sql b/src/test/regress/sql/multi_mx_reference_table.sql index 35b18ad39..ea43da9bf 100644 --- a/src/test/regress/sql/multi_mx_reference_table.sql +++ b/src/test/regress/sql/multi_mx_reference_table.sql @@ -536,7 +536,7 @@ WHERE colocated_table_test.value_1 = colocated_table_test_2.value_1 AND colocated_table_test.value_2 = reference_table_test.value_2 ORDER BY 1; -SET citus.task_executor_type to "task-tracker"; +SET citus.enable_repartition_joins to ON; SELECT colocated_table_test.value_2 FROM diff --git a/src/test/regress/sql/multi_mx_repartition_join_w1.sql b/src/test/regress/sql/multi_mx_repartition_join_w1.sql index a9e0b6e10..6cc162179 100644 --- a/src/test/regress/sql/multi_mx_repartition_join_w1.sql +++ b/src/test/regress/sql/multi_mx_repartition_join_w1.sql @@ -4,7 +4,8 @@ -- tests use the same sequence ids but the queries should not fail. \c - - - :worker_1_port -SET citus.task_executor_type TO "task-tracker"; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; CREATE TEMP TABLE t1 AS SELECT l1.l_comment diff --git a/src/test/regress/sql/multi_mx_repartition_join_w2.sql b/src/test/regress/sql/multi_mx_repartition_join_w2.sql index d27fb57a2..a5c0d4ce9 100644 --- a/src/test/regress/sql/multi_mx_repartition_join_w2.sql +++ b/src/test/regress/sql/multi_mx_repartition_join_w2.sql @@ -4,7 +4,8 @@ -- tests use the same sequence ids but the queries should not fail. \c - - - :worker_2_port -SET citus.task_executor_type TO "task-tracker"; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; CREATE TEMP TABLE t1 AS SELECT l1.l_comment diff --git a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql index 6bc5c16d6..bce22ac2c 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_prepare.sql @@ -183,6 +183,7 @@ INSERT INTO repartition_udt_other values (12, '(2,3)'::test_udt, 'foo'); SET client_min_messages = LOG; SET citus.task_executor_type = 'task-tracker'; +SET citus.enable_repartition_joins to ON; -- Query that should result in a repartition -- join on int column, and be empty. diff --git a/src/test/regress/sql/multi_mx_repartition_udt_w1.sql b/src/test/regress/sql/multi_mx_repartition_udt_w1.sql index 0710d98b4..574528550 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_w1.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_w1.sql @@ -5,7 +5,8 @@ \c - - - :worker_1_port SET client_min_messages = LOG; -- Query that should result in a repartition join on UDT column. -SET citus.task_executor_type = 'task-tracker'; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; SET citus.log_multi_join_order = true; -- Query that should result in a repartition diff --git a/src/test/regress/sql/multi_mx_repartition_udt_w2.sql b/src/test/regress/sql/multi_mx_repartition_udt_w2.sql index fd39c8b86..30bde9bf1 100644 --- a/src/test/regress/sql/multi_mx_repartition_udt_w2.sql +++ b/src/test/regress/sql/multi_mx_repartition_udt_w2.sql @@ -5,7 +5,8 @@ \c - - - :worker_2_port SET client_min_messages = LOG; -- Query that should result in a repartition join on UDT column. -SET citus.task_executor_type = 'task-tracker'; +SET citus.max_adaptive_executor_pool_size TO 1; +SET citus.enable_repartition_joins to ON; SET citus.log_multi_join_order = true; -- Query that should result in a repartition diff --git a/src/test/regress/sql/multi_mx_router_planner.sql b/src/test/regress/sql/multi_mx_router_planner.sql index fe7cb54ee..1644c2929 100644 --- a/src/test/regress/sql/multi_mx_router_planner.sql +++ b/src/test/regress/sql/multi_mx_router_planner.sql @@ -66,7 +66,6 @@ INSERT INTO articles_hash_mx VALUES (48, 8, 'alkylic', 18610); INSERT INTO articles_hash_mx VALUES (49, 9, 'anyone', 2681); INSERT INTO articles_hash_mx VALUES (50, 10, 'anjanette', 19519); -RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- insert a single row for the test @@ -388,7 +387,6 @@ SELECT * WHERE author_id >= 1 AND author_id <= 3 ORDER BY 1,2,3,4; -RESET citus.task_executor_type; -- Test various filtering options for router plannable check SET client_min_messages to 'DEBUG2'; @@ -647,8 +645,7 @@ CREATE MATERIALIZED VIEW mv_articles_hash_mx_error AS -- following query is router plannable, but router planner is disabled -- TODO: Uncomment once we fix task-tracker issue ---SET citus.task_executor_type to 'task-tracker'; ---SELECT id +----SELECT id -- FROM articles_hash_mx -- WHERE author_id = 1; diff --git a/src/test/regress/sql/multi_mx_schema_support.sql b/src/test/regress/sql/multi_mx_schema_support.sql index 40f6c10e4..168aede74 100644 --- a/src/test/regress/sql/multi_mx_schema_support.sql +++ b/src/test/regress/sql/multi_mx_schema_support.sql @@ -101,7 +101,7 @@ SELECT * FROM citus_mx_test_schema.nation_hash_composite_types WHERE test_col = SET search_path TO citus_mx_test_schema; SELECT * FROM nation_hash_composite_types WHERE test_col = '(a,a)'::new_composite_type ORDER BY 1::int DESC; - +SET citus.enable_repartition_joins to ON; -- check when search_path is public, -- join of two tables which are in different schemas, -- join on partition column @@ -147,7 +147,6 @@ WHERE n1.n_nationkey = n2.n_nationkey; -- single repartition joins -SET citus.task_executor_type TO "task-tracker"; -- check when search_path is public, -- join of two tables which are in different schemas, @@ -218,7 +217,6 @@ WHERE n1.n_regionkey = n2.n_regionkey; -- set task_executor back to adaptive -SET citus.task_executor_type TO "adaptive"; -- connect to the master and do some test -- regarding DDL support on schemas where diff --git a/src/test/regress/sql/multi_null_minmax_value_pruning.sql b/src/test/regress/sql/multi_null_minmax_value_pruning.sql index 843898daa..eb2fb3eae 100644 --- a/src/test/regress/sql/multi_null_minmax_value_pruning.sql +++ b/src/test/regress/sql/multi_null_minmax_value_pruning.sql @@ -10,7 +10,6 @@ SET client_min_messages TO DEBUG2; SET citus.explain_all_tasks TO on; -- to avoid differing explain output - executor doesn't matter, -- because were testing pruning here. -RESET citus.task_executor_type; -- Change configuration to treat lineitem and orders tables as large diff --git a/src/test/regress/sql/multi_partitioning.sql b/src/test/regress/sql/multi_partitioning.sql index e0297e54f..e9d8e944d 100644 --- a/src/test/regress/sql/multi_partitioning.sql +++ b/src/test/regress/sql/multi_partitioning.sql @@ -5,6 +5,7 @@ SET citus.next_shard_id TO 1660000; SET citus.shard_count TO 4; SET citus.shard_replication_factor TO 1; +SET citus.enable_repartition_joins to ON; -- -- Distributed Partitioned Table Creation Tests @@ -863,12 +864,10 @@ SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass COMMIT; -- test locks on task-tracker SELECT -SET citus.task_executor_type TO 'task-tracker'; BEGIN; SELECT * FROM partitioning_locks AS pl1 JOIN partitioning_locks AS pl2 ON pl1.id = pl2.ref_id ORDER BY 1, 2; SELECT relation::regclass, locktype, mode FROM pg_locks WHERE relation::regclass::text LIKE 'partitioning_locks%' AND pid = pg_backend_pid() ORDER BY 1, 2, 3; COMMIT; -RESET citus.task_executor_type; -- test locks on INSERT BEGIN; diff --git a/src/test/regress/sql/multi_prepare_plsql.sql b/src/test/regress/sql/multi_prepare_plsql.sql index 8e1892ad1..85542aaf3 100644 --- a/src/test/regress/sql/multi_prepare_plsql.sql +++ b/src/test/regress/sql/multi_prepare_plsql.sql @@ -147,7 +147,6 @@ BEGIN END; $$ LANGUAGE plpgsql; -SET citus.task_executor_type TO 'task-tracker'; SET client_min_messages TO INFO; -- now, run PL/pgsql functions @@ -176,7 +175,6 @@ SELECT plpgsql_test_2(); -- run the tests which do not require re-partition -- with real-time executor -RESET citus.task_executor_type; -- now, run PL/pgsql functions SELECT plpgsql_test_1(); @@ -365,7 +363,6 @@ SELECT real_time_partition_column_select(5); SELECT real_time_partition_column_select(6); -- check task-tracker executor -SET citus.task_executor_type TO 'task-tracker'; CREATE FUNCTION task_tracker_non_partition_column_select(value_arg int) RETURNS TABLE(key int, value int) AS $$ @@ -420,7 +417,6 @@ SELECT task_tracker_partition_column_select(4); SELECT task_tracker_partition_column_select(5); SELECT task_tracker_partition_column_select(6); -RESET citus.task_executor_type; -- check updates CREATE FUNCTION partition_parameter_update(int, int) RETURNS void as $$ diff --git a/src/test/regress/sql/multi_prepare_sql.sql b/src/test/regress/sql/multi_prepare_sql.sql index d6c1762b8..d964a50f6 100644 --- a/src/test/regress/sql/multi_prepare_sql.sql +++ b/src/test/regress/sql/multi_prepare_sql.sql @@ -110,7 +110,6 @@ ORDER BY cust_nation, l_year; -SET citus.task_executor_type TO 'task-tracker'; SET client_min_messages TO INFO; -- execute prepared statements @@ -142,7 +141,6 @@ CREATE TEMP TABLE prepared_sql_test_7 AS EXECUTE prepared_test_7('UNITED KINGDOM SELECT * from prepared_sql_test_7; -- now, run some of the tests with real-time executor -RESET citus.task_executor_type; -- execute prepared statements EXECUTE prepared_test_1; @@ -396,7 +394,6 @@ EXECUTE prepared_real_time_partition_column_select(5); EXECUTE prepared_real_time_partition_column_select(6); -- check task-tracker executor -SET citus.task_executor_type TO 'task-tracker'; PREPARE prepared_task_tracker_non_partition_column_select(int) AS SELECT @@ -437,7 +434,6 @@ EXECUTE prepared_task_tracker_partition_column_select(4); EXECUTE prepared_task_tracker_partition_column_select(5); EXECUTE prepared_task_tracker_partition_column_select(6); -RESET citus.task_executor_type; -- check updates PREPARE prepared_partition_parameter_update(int, int) AS diff --git a/src/test/regress/sql/multi_reference_table.sql b/src/test/regress/sql/multi_reference_table.sql index 39d75ef06..aec36920b 100644 --- a/src/test/regress/sql/multi_reference_table.sql +++ b/src/test/regress/sql/multi_reference_table.sql @@ -677,26 +677,26 @@ WHERE colocated_table_test.value_1 = colocated_table_test_2.value_1 AND colocated_table_test.value_2 = reference_table_test.value_2 ORDER BY 1; -SET citus.task_executor_type to "task-tracker"; +SET citus.enable_repartition_joins to ON; SELECT colocated_table_test.value_2 FROM reference_table_test, colocated_table_test, colocated_table_test_2 WHERE - colocated_table_test.value_2 = colocated_table_test_2.value_2 AND colocated_table_test.value_2 = reference_table_test.value_2; + colocated_table_test.value_2 = colocated_table_test_2.value_2 AND colocated_table_test.value_2 = reference_table_test.value_2 +ORDER BY colocated_table_test.value_2; SELECT reference_table_test.value_2 FROM reference_table_test, colocated_table_test, colocated_table_test_2 WHERE - colocated_table_test.value_1 = reference_table_test.value_1 AND colocated_table_test_2.value_1 = reference_table_test.value_1; - + colocated_table_test.value_1 = reference_table_test.value_1 AND colocated_table_test_2.value_1 = reference_table_test.value_1 +ORDER BY reference_table_test.value_2; SET citus.log_multi_join_order TO FALSE; SET citus.shard_count TO DEFAULT; -SET citus.task_executor_type to "adaptive"; -- some INSERT .. SELECT queries that involve both hash distributed and reference tables diff --git a/src/test/regress/sql/multi_repartition_join_planning.sql b/src/test/regress/sql/multi_repartition_join_planning.sql index a7bd65e7f..872a49c83 100644 --- a/src/test/regress/sql/multi_repartition_join_planning.sql +++ b/src/test/regress/sql/multi_repartition_join_planning.sql @@ -9,6 +9,7 @@ SET citus.next_shard_id TO 690000; SET citus.enable_unique_job_ids TO off; +SET citus.enable_repartition_joins to ON; create schema repartition_join; DROP TABLE IF EXISTS repartition_join.order_line; @@ -34,8 +35,7 @@ SELECT create_distributed_table('order_line','ol_w_id'); SELECT create_distributed_table('stock','s_w_id'); BEGIN; -SET client_min_messages TO DEBUG4; -SET citus.task_executor_type TO 'task-tracker'; +SET client_min_messages TO DEBUG; -- Debug4 log messages display jobIds within them. We explicitly set the jobId -- sequence here so that the regression output becomes independent of the number diff --git a/src/test/regress/sql/multi_repartition_join_pruning.sql b/src/test/regress/sql/multi_repartition_join_pruning.sql index 0e8090e02..9ff0be269 100644 --- a/src/test/regress/sql/multi_repartition_join_pruning.sql +++ b/src/test/regress/sql/multi_repartition_join_pruning.sql @@ -10,7 +10,7 @@ SET citus.next_shard_id TO 700000; SET client_min_messages TO DEBUG2; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; -- Single range-repartition join to test join-pruning behaviour. EXPLAIN (COSTS OFF) diff --git a/src/test/regress/sql/multi_repartition_join_task_assignment.sql b/src/test/regress/sql/multi_repartition_join_task_assignment.sql index f7c651bda..be100c2fc 100644 --- a/src/test/regress/sql/multi_repartition_join_task_assignment.sql +++ b/src/test/regress/sql/multi_repartition_join_task_assignment.sql @@ -11,7 +11,7 @@ SET citus.next_shard_id TO 710000; BEGIN; SET client_min_messages TO DEBUG3; -SET citus.task_executor_type TO 'task-tracker'; +SET citus.enable_repartition_joins to ON; -- Single range repartition join to test anchor-shard based task assignment and -- assignment propagation to merge and data-fetch tasks. diff --git a/src/test/regress/sql/multi_repartition_udt.sql b/src/test/regress/sql/multi_repartition_udt.sql index d0797b9a0..d4f029929 100644 --- a/src/test/regress/sql/multi_repartition_udt.sql +++ b/src/test/regress/sql/multi_repartition_udt.sql @@ -3,6 +3,7 @@ -- SET citus.next_shard_id TO 535000; +SET citus.enable_repartition_joins to ON; -- START type creation @@ -160,6 +161,7 @@ FUNCTION 1 test_udt_hash(test_udt); -- Distribute and populate the two tables. SET citus.shard_count TO 3; SET citus.shard_replication_factor TO 1; +SET citus.enable_repartition_joins to ON; SELECT create_distributed_table('repartition_udt', 'pk', 'hash'); SET citus.shard_count TO 5; SELECT create_distributed_table('repartition_udt_other', 'pk', 'hash'); diff --git a/src/test/regress/sql/multi_repartitioned_subquery_udf.sql b/src/test/regress/sql/multi_repartitioned_subquery_udf.sql index e3cb682bb..ed432b797 100644 --- a/src/test/regress/sql/multi_repartitioned_subquery_udf.sql +++ b/src/test/regress/sql/multi_repartitioned_subquery_udf.sql @@ -43,7 +43,6 @@ $_$; -- Run query on master \c - - :master_host :master_port -SET citus.task_executor_type TO 'task-tracker'; SELECT * FROM (SELECT median(ARRAY[1,2,sum(l_suppkey)]) as median, count(*) FROM lineitem GROUP BY l_partkey) AS a diff --git a/src/test/regress/sql/multi_router_planner.sql b/src/test/regress/sql/multi_router_planner.sql index a3f8fe2d2..4f10a1166 100644 --- a/src/test/regress/sql/multi_router_planner.sql +++ b/src/test/regress/sql/multi_router_planner.sql @@ -111,7 +111,6 @@ INSERT INTO articles_hash VALUES (50, 10, 'anjanette', 19519); -RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- insert a single row for the test @@ -507,7 +506,6 @@ SELECT * WHERE author_id >= 1 AND author_id <= 3 ORDER BY 1,2,3,4; -RESET citus.task_executor_type; -- Test various filtering options for router plannable check SET client_min_messages to 'DEBUG2'; @@ -795,10 +793,10 @@ SELECT master_create_empty_shard('authors_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 1, shardmaxvalue=10 WHERE shardid = :shard_id; SELECT master_create_empty_shard('authors_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=30 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=20 WHERE shardid = :shard_id; SELECT master_create_empty_shard('authors_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=40 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=30 WHERE shardid = :shard_id; SELECT master_create_empty_shard('authors_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 31, shardmaxvalue=40 WHERE shardid = :shard_id; @@ -807,10 +805,10 @@ SELECT master_create_empty_shard('articles_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 1, shardmaxvalue=10 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=30 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 11, shardmaxvalue=20 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset -UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=40 WHERE shardid = :shard_id; +UPDATE pg_dist_shard SET shardminvalue = 21, shardmaxvalue=30 WHERE shardid = :shard_id; SELECT master_create_empty_shard('articles_range') as shard_id \gset UPDATE pg_dist_shard SET shardminvalue = 31, shardmaxvalue=40 WHERE shardid = :shard_id; @@ -836,7 +834,7 @@ RESET citus.log_remote_commands; -- This query was intended to test "multi-shard join is not router plannable" -- To run it using repartition join logic we change the join columns -SET citus.task_executor_type to "task-tracker"; +SET citus.enable_repartition_joins to ON; SELECT * FROM articles_range ar join authors_range au on (ar.title = au.name) WHERE ar.author_id = 35; @@ -845,7 +843,6 @@ SELECT * FROM articles_range ar join authors_range au on (ar.title = au.name) -- change the join columns. SELECT * FROM articles_range ar join authors_range au on (ar.title = au.name) WHERE ar.author_id = 1 or au.id = 5; -RESET citus.task_executor_type; -- bogus query, join on non-partition column, but router plannable due to filters SELECT * FROM articles_range ar join authors_range au on (ar.id = au.id) @@ -861,7 +858,6 @@ SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id) -- not router plannable SELECT * FROM articles_hash ar join authors_range au on (ar.author_id = au.id) WHERE ar.author_id = 3; - -- join between a range partitioned table and reference table is router plannable SELECT * FROM articles_range ar join authors_reference au on (ar.author_id = au.id) WHERE ar.author_id = 1; @@ -1116,7 +1112,6 @@ CREATE MATERIALIZED VIEW mv_articles_hash_data AS SELECT * FROM mv_articles_hash_data ORDER BY 1, 2, 3, 4; -- router planner/executor is now enabled for task-tracker executor -SET citus.task_executor_type to 'task-tracker'; SELECT id FROM articles_hash WHERE author_id = 1 diff --git a/src/test/regress/sql/multi_router_planner_fast_path.sql b/src/test/regress/sql/multi_router_planner_fast_path.sql index 542ead437..99292b00d 100644 --- a/src/test/regress/sql/multi_router_planner_fast_path.sql +++ b/src/test/regress/sql/multi_router_planner_fast_path.sql @@ -67,7 +67,6 @@ INSERT INTO articles_hash VALUES (1, 1, 'arsenous', 9572), (2, 2, 'abducing', (44, 4, 'anteport', 16793),(45, 5, 'afrasia', 864),(46, 6, 'atlanta', 17702),(47, 7, 'abeyance', 1772), (48, 8, 'alkylic', 18610),(49, 9, 'anyone', 2681),(50, 10, 'anjanette', 19519); -RESET citus.task_executor_type; SET client_min_messages TO 'DEBUG2'; -- test simple select for a single row @@ -807,7 +806,6 @@ SELECT * FROM mv_articles_hash_empty; -- fast-path router planner/executor is enabled for task-tracker executor -SET citus.task_executor_type to 'task-tracker'; SELECT id FROM articles_hash WHERE author_id = 1; diff --git a/src/test/regress/sql/multi_schema_support.sql b/src/test/regress/sql/multi_schema_support.sql index b08d59849..698e8111f 100644 --- a/src/test/regress/sql/multi_schema_support.sql +++ b/src/test/regress/sql/multi_schema_support.sql @@ -601,6 +601,7 @@ SELECT create_distributed_table('test_schema_support_join_2.nation_hash', 'n_nat 5|ETHIOPIA|0|ven packages wake quickly. regu \. +SET citus.enable_repartition_joins to ON; -- check when search_path is public, -- join of two tables which are in different schemas, -- join on partition column @@ -646,7 +647,6 @@ WHERE n1.n_nationkey = n2.n_nationkey; -- single repartition joins -SET citus.task_executor_type TO "task-tracker"; -- check when search_path is public, -- join of two tables which are in different schemas, @@ -717,7 +717,6 @@ WHERE n1.n_regionkey = n2.n_regionkey; -- set task_executor back to adaptive -SET citus.task_executor_type TO "adaptive"; -- test ALTER TABLE SET SCHEMA @@ -904,7 +903,6 @@ SELECT sum(result::int) FROM run_command_on_placements('run_test_schema.test_tab SELECT sum(result::int) FROM run_command_on_shards('run_test_schema.test_table','SELECT pg_table_size(''%s'')'); -- test capital letters on both table and schema names -SET citus.task_executor_type to "adaptive"; -- create schema with weird names CREATE SCHEMA "CiTuS.TeeN"; CREATE SCHEMA "CiTUS.TEEN2"; diff --git a/src/test/regress/sql/multi_simple_queries.sql b/src/test/regress/sql/multi_simple_queries.sql index be2558325..a95f93f68 100644 --- a/src/test/regress/sql/multi_simple_queries.sql +++ b/src/test/regress/sql/multi_simple_queries.sql @@ -218,7 +218,6 @@ SELECT o_orderstatus, sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders -- now, test the cases where Citus do or do not need to create -- the master queries SET client_min_messages TO 'DEBUG2'; -SET citus.task_executor_type TO 'adaptive'; -- start with the simple lookup query SELECT * diff --git a/src/test/regress/sql/multi_single_relation_subquery.sql b/src/test/regress/sql/multi_single_relation_subquery.sql index c40fa37d3..01bf5cba5 100644 --- a/src/test/regress/sql/multi_single_relation_subquery.sql +++ b/src/test/regress/sql/multi_single_relation_subquery.sql @@ -7,7 +7,6 @@ SET citus.next_shard_id TO 860000; -SET citus.task_executor_type TO 'task-tracker'; select number_sum, diff --git a/src/test/regress/sql/multi_sql_function.sql b/src/test/regress/sql/multi_sql_function.sql index 4b882608f..329d57996 100644 --- a/src/test/regress/sql/multi_sql_function.sql +++ b/src/test/regress/sql/multi_sql_function.sql @@ -40,7 +40,6 @@ CREATE FUNCTION sql_test_no_4() RETURNS bigint AS ' o_orderkey = l_orderkey; ' LANGUAGE SQL; -SET citus.task_executor_type TO 'task-tracker'; SET client_min_messages TO INFO; -- now, run plain SQL functions @@ -51,7 +50,6 @@ SELECT sql_test_no_4(); -- run the tests which do not require re-partition -- with real-time executor -RESET citus.task_executor_type; -- now, run plain SQL functions SELECT sql_test_no_1(); diff --git a/src/test/regress/sql/multi_task_string_size.sql b/src/test/regress/sql/multi_task_string_size.sql index a12a88dc6..969e4ccb4 100644 --- a/src/test/regress/sql/multi_task_string_size.sql +++ b/src/test/regress/sql/multi_task_string_size.sql @@ -209,7 +209,6 @@ CREATE TABLE wide_table SELECT create_distributed_table('wide_table', 'long_column_001'); -SET citus.task_executor_type TO 'task-tracker'; SHOW citus.max_task_string_size; @@ -225,6 +224,7 @@ SELECT raise_failed_execution(' SELECT u.* FROM wide_table u JOIN wide_table v ON (u.long_column_002 = v.long_column_003); '); +SET citus.enable_repartition_joins to ON; -- following will succeed since it fetches few columns SELECT u.long_column_001, u.long_column_002, u.long_column_003 FROM wide_table u JOIN wide_table v ON (u.long_column_002 = v.long_column_003); @@ -233,5 +233,4 @@ RESET client_min_messages; DROP TABLE wide_table; RESET citus.shard_count; -RESET citus.task_executor_type; diff --git a/src/test/regress/sql/multi_view.sql b/src/test/regress/sql/multi_view.sql index b529878ec..057cbd907 100644 --- a/src/test/regress/sql/multi_view.sql +++ b/src/test/regress/sql/multi_view.sql @@ -85,7 +85,6 @@ RESET client_min_messages; SELECT count(*) FROM priority_orders JOIN air_shipped_lineitems ON (o_custkey = l_suppkey); -SET citus.task_executor_type to "task-tracker"; -- single view repartition subqueries are not supported SELECT l_suppkey, count(*) FROM diff --git a/src/test/regress/sql/propagate_set_commands.sql b/src/test/regress/sql/propagate_set_commands.sql index 8fb987468..3c7bef3b0 100644 --- a/src/test/regress/sql/propagate_set_commands.sql +++ b/src/test/regress/sql/propagate_set_commands.sql @@ -9,7 +9,6 @@ INSERT INTO test VALUES (1,1), (3,3); SET citus.propagate_set_commands TO 'local'; -- make sure we send BEGIN before a SELECT -SET citus.task_executor_type TO 'adaptive'; SET citus.select_opens_transaction_block TO on; BEGIN; diff --git a/src/test/regress/sql/set_operation_and_local_tables.sql b/src/test/regress/sql/set_operation_and_local_tables.sql index b00fd67d2..8930cce87 100644 --- a/src/test/regress/sql/set_operation_and_local_tables.sql +++ b/src/test/regress/sql/set_operation_and_local_tables.sql @@ -1,6 +1,8 @@ CREATE SCHEMA recursive_set_local; SET search_path TO recursive_set_local, public; +SET citus.enable_repartition_joins to ON; + CREATE TABLE recursive_set_local.test (x int, y int); SELECT create_distributed_table('test', 'x'); @@ -93,12 +95,10 @@ SELECT * FROM ((SELECT * FROM local_test) INTERSECT (SELECT * FROM test ORDER BY -- set operations and the sublink can be recursively planned SELECT * FROM ((SELECT x FROM test) UNION (SELECT x FROM (SELECT x FROM local_test) as foo WHERE x IN (SELECT x FROM test))) u ORDER BY 1; -SET citus.task_executor_type TO 'task-tracker'; -- repartition is recursively planned before the set operation (SELECT x FROM test) INTERSECT (SELECT t1.x FROM test as t1, test as t2 WHERE t1.x = t2.y LIMIT 2) INTERSECT (((SELECT x FROM local_test) UNION ALL (SELECT x FROM test)) INTERSECT (SELECT i FROM generate_series(0, 100) i)) ORDER BY 1 DESC; -SET citus.task_executor_type TO 'adaptive'; RESET client_min_messages; DROP SCHEMA recursive_set_local CASCADE; diff --git a/src/test/regress/sql/task_tracker_assign_task.sql b/src/test/regress/sql/task_tracker_assign_task.sql deleted file mode 100644 index d19956773..000000000 --- a/src/test/regress/sql/task_tracker_assign_task.sql +++ /dev/null @@ -1,51 +0,0 @@ --- --- TASK_TRACKER_ASSIGN_TASK --- - - - -\set JobId 401010 -\set SimpleTaskId 101101 -\set RecoverableTaskId 801102 - -\set SimpleTaskTable lineitem_simple_task -\set BadQueryString '\'SELECT COUNT(*) FROM bad_table_name\'' -\set GoodQueryString '\'SELECT COUNT(*) FROM lineitem\'' -\set SelectAll 'SELECT *' - --- We assign two tasks to the task tracker. The first task simply executes. The --- recoverable task on the other hand repeatedly fails, and we sleep until the --- task tracker stops retrying the recoverable task. - -SELECT task_tracker_assign_task(:JobId, :SimpleTaskId, - 'COPY (SELECT * FROM lineitem) TO ' - '''base/pgsql_job_cache/job_401010/task_101101'''); - -SELECT task_tracker_assign_task(:JobId, :RecoverableTaskId, :BadQueryString); - --- After assigning the two tasks, we wait for them to make progress. Note that --- these tasks get scheduled and run asynchronously, so if the sleep interval is --- not enough, the regression tests may fail on an overloaded box. - -SELECT pg_sleep(3.0); - -SELECT task_tracker_task_status(:JobId, :SimpleTaskId); -SELECT task_tracker_task_status(:JobId, :RecoverableTaskId); - -COPY :SimpleTaskTable FROM 'base/pgsql_job_cache/job_401010/task_101101'; - -SELECT COUNT(*) FROM :SimpleTaskTable; - -SELECT COUNT(*) AS diff_lhs FROM ( :SelectAll FROM :SimpleTaskTable EXCEPT ALL - :SelectAll FROM lineitem ) diff; -SELECT COUNT(*) As diff_rhs FROM ( :SelectAll FROM lineitem EXCEPT ALL - :SelectAll FROM :SimpleTaskTable ) diff; - --- We now reassign the recoverable task with a good query string. This updates --- the task's query string, and reschedules the updated task for execution. - -SELECT task_tracker_assign_task(:JobId, :RecoverableTaskId, :GoodQueryString); - -SELECT pg_sleep(2.0); - -SELECT task_tracker_task_status(:JobId, :RecoverableTaskId); diff --git a/src/test/regress/sql/task_tracker_cleanup_job.sql b/src/test/regress/sql/task_tracker_cleanup_job.sql deleted file mode 100644 index ffabf1eed..000000000 --- a/src/test/regress/sql/task_tracker_cleanup_job.sql +++ /dev/null @@ -1,51 +0,0 @@ --- --- TASK_TRACKER_CLEANUP_JOB --- - - -SET citus.next_shard_id TO 1060000; - - -\set JobId 401010 -\set CompletedTaskId 801107 -\set RunningTaskId 801108 - --- Test worker_cleanup_job_schema_cache -SELECT * FROM task_tracker_assign_task(2, 2, ''); -SELECT count(*) FROM pg_catalog.pg_namespace WHERE nspname = 'pg_merge_job_0002'; -SELECT worker_cleanup_job_schema_cache(); -SELECT count(*) FROM pg_catalog.pg_namespace WHERE nspname = 'pg_merge_job_0002'; - --- We assign two tasks to the task tracker. The first task should complete and --- the second task should continue to keep running. - -SELECT task_tracker_assign_task(:JobId, :CompletedTaskId, - 'COPY (SELECT * FROM lineitem) TO ' - '''base/pgsql_job_cache/job_401010/task_801107'''); - -SELECT task_tracker_assign_task(:JobId, :RunningTaskId, - 'SELECT pg_sleep(100)'); - -SELECT pg_sleep(2.0); - -SELECT task_tracker_task_status(:JobId, :CompletedTaskId); -SELECT task_tracker_task_status(:JobId, :RunningTaskId); - -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010/task_801107'); -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010'); - --- We now clean up all tasks for this job id. As a result, shared hash entries, --- files, and connections associated with these tasks should all be cleaned up. - -SELECT task_tracker_cleanup_job(:JobId); - -SELECT pg_sleep(1.0); - -SELECT task_tracker_task_status(:JobId, :CompletedTaskId); -SELECT task_tracker_task_status(:JobId, :RunningTaskId); - -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010/task_801107'); -SELECT isdir FROM pg_stat_file('base/pgsql_job_cache/job_401010'); - --- Also clean up worker_cleanup_job_schema_cache job -SELECT task_tracker_cleanup_job(2); diff --git a/src/test/regress/sql/task_tracker_create_table.sql b/src/test/regress/sql/task_tracker_create_table.sql deleted file mode 100644 index c2e474bc9..000000000 --- a/src/test/regress/sql/task_tracker_create_table.sql +++ /dev/null @@ -1,17 +0,0 @@ --- --- TASK_TRACKER_CREATE_TABLE --- - - -SET citus.next_shard_id TO 1070000; - - --- New table definitions to test the task tracker process and protocol - -CREATE TABLE lineitem_simple_task ( LIKE lineitem ); -CREATE TABLE lineitem_compute_task ( LIKE lineitem ); -CREATE TABLE lineitem_compute_update_task ( LIKE lineitem ); - -CREATE TABLE lineitem_partition_task_part_00 ( LIKE lineitem ); -CREATE TABLE lineitem_partition_task_part_01 ( LIKE lineitem ); -CREATE TABLE lineitem_partition_task_part_02 ( LIKE lineitem ); diff --git a/src/test/regress/sql/task_tracker_partition_task.sql b/src/test/regress/sql/task_tracker_partition_task.sql deleted file mode 100644 index 89fc0bbe2..000000000 --- a/src/test/regress/sql/task_tracker_partition_task.sql +++ /dev/null @@ -1,69 +0,0 @@ --- --- TASK_TRACKER_PARTITION_TASK --- - - - -\set JobId 401010 -\set PartitionTaskId 801106 - -\set PartitionColumn l_orderkey -\set SelectAll 'SELECT *' - -\set TablePart00 lineitem_partition_task_part_00 -\set TablePart01 lineitem_partition_task_part_01 -\set TablePart02 lineitem_partition_task_part_02 - -SELECT usesysid AS userid FROM pg_user WHERE usename = current_user \gset - -\set File_Basedir base/pgsql_job_cache -\set Table_File_00 :File_Basedir/job_:JobId/task_:PartitionTaskId/p_00000.:userid -\set Table_File_01 :File_Basedir/job_:JobId/task_:PartitionTaskId/p_00001.:userid -\set Table_File_02 :File_Basedir/job_:JobId/task_:PartitionTaskId/p_00002.:userid - --- We assign a partition task and wait for it to complete. Note that we hardcode --- the partition function call string, including the job and task identifiers, --- into the argument in the task assignment function. This hardcoding is --- necessary as the current psql version does not perform variable interpolation --- for names inside single quotes. - -SELECT task_tracker_assign_task(:JobId, :PartitionTaskId, - 'SELECT worker_range_partition_table(' - '401010, 801106, ''SELECT * FROM lineitem'', ' - '''l_orderkey'', 20, ARRAY[1000, 3000]::_int8)'); - -SELECT pg_sleep(4.0); - -SELECT task_tracker_task_status(:JobId, :PartitionTaskId); - -COPY :TablePart00 FROM :'Table_File_00'; -COPY :TablePart01 FROM :'Table_File_01'; -COPY :TablePart02 FROM :'Table_File_02'; - -SELECT COUNT(*) FROM :TablePart00; -SELECT COUNT(*) FROM :TablePart02; - --- We first compute the difference of partition tables against the base table. --- Then, we compute the difference of the base table against partitioned tables. - -SELECT COUNT(*) AS diff_lhs_00 FROM ( - :SelectAll FROM :TablePart00 EXCEPT ALL - :SelectAll FROM lineitem WHERE :PartitionColumn < 1000 ) diff; -SELECT COUNT(*) AS diff_lhs_01 FROM ( - :SelectAll FROM :TablePart01 EXCEPT ALL - :SelectAll FROM lineitem WHERE :PartitionColumn >= 1000 AND - :PartitionColumn < 3000 ) diff; -SELECT COUNT(*) AS diff_lhs_02 FROM ( - :SelectAll FROM :TablePart02 EXCEPT ALL - :SelectAll FROM lineitem WHERE :PartitionColumn >= 3000 ) diff; - -SELECT COUNT(*) AS diff_rhs_00 FROM ( - :SelectAll FROM lineitem WHERE :PartitionColumn < 1000 EXCEPT ALL - :SelectAll FROM :TablePart00 ) diff; -SELECT COUNT(*) AS diff_rhs_01 FROM ( - :SelectAll FROM lineitem WHERE :PartitionColumn >= 1000 AND - :PartitionColumn < 3000 EXCEPT ALL - :SelectAll FROM :TablePart01 ) diff; -SELECT COUNT(*) AS diff_rhs_02 FROM ( - :SelectAll FROM lineitem WHERE :PartitionColumn >= 3000 EXCEPT ALL - :SelectAll FROM :TablePart02 ) diff; diff --git a/src/test/regress/sql/worker_check_invalid_arguments.sql b/src/test/regress/sql/worker_check_invalid_arguments.sql deleted file mode 100644 index ee6da8c7c..000000000 --- a/src/test/regress/sql/worker_check_invalid_arguments.sql +++ /dev/null @@ -1,105 +0,0 @@ --- --- WORKER_CHECK_INVALID_ARGUMENTS --- - - -SET citus.next_shard_id TO 1100000; - - -\set JobId 201010 -\set TaskId 101108 -\set Table_Name simple_binary_data_table -\set Partition_Column_Name '\'textcolumn\'' -\set Partition_Column_Type 25 -\set Partition_Count 2 -\set Select_Query_Text '\'SELECT * FROM simple_binary_data_table\'' - -\set Bad_Partition_Column_Name '\'badcolumnname\'' -\set Bad_Partition_Column_Type 20 -\set Bad_Select_Query_Text '\'SELECT * FROM bad_table_name\'' - --- Create simple table and insert a few rows into this table --- N.B. - These rows will be partitioned to files on disk then read back in the --- order the files are listed by a call to readdir; because this order is not --- predictable, the second column of these rows always has the same value, to --- avoid an error message differing based on file read order. - -CREATE TABLE :Table_Name(textcolumn text, binarycolumn bytea); -COPY :Table_Name FROM stdin; -aaa \013\120 -some\t tabs\t with \t spaces \013\120 -\. - -SELECT COUNT(*) FROM :Table_Name; - --- Check that we fail with bad SQL query - -SELECT worker_range_partition_table(:JobId, :TaskId, :Bad_Select_Query_Text, - :Partition_Column_Name, :Partition_Column_Type, - ARRAY['aaa', 'some']::_text); - --- Check that we fail with bad partition column name - -SELECT worker_range_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Bad_Partition_Column_Name, :Partition_Column_Type, - ARRAY['aaa', 'some']::_text); - --- Check that we fail when partition column and split point types do not match - -SELECT worker_range_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Partition_Column_Name, :Bad_Partition_Column_Type, - ARRAY['aaa', 'some']::_text); - --- Check that we fail with bad partition column type on hash partitioning - -SELECT worker_hash_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Partition_Column_Name, :Bad_Partition_Column_Type, - ARRAY[-2147483648, -1073741824, 0, 1073741824]::int4[]); - --- Now, partition table data using valid arguments - -SELECT worker_range_partition_table(:JobId, :TaskId, :Select_Query_Text, - :Partition_Column_Name, :Partition_Column_Type, - ARRAY['aaa', 'some']::_text); - --- Check that we fail to merge when the number of column names and column types --- do not match - -SELECT worker_merge_files_into_table(:JobId, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea', 'integer']); - --- Check that we fail to merge when column types do not match underlying data - -SELECT worker_merge_files_into_table(:JobId, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'integer']); - --- Check that we fail to merge when ids are wrong - -SELECT worker_merge_files_into_table(-1, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea']); - -SELECT worker_merge_files_into_table(:JobId, -1, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea']); - -SELECT worker_merge_files_and_run_query(-1, :TaskId, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); - -SELECT worker_merge_files_and_run_query(:JobId, -1, - 'CREATE TABLE task_000001_merge(merge_column_0 int)', - 'CREATE TABLE task_000001 (a) AS SELECT sum(merge_column_0) FROM task_000001_merge' -); - --- Finally, merge partitioned files using valid arguments - -SELECT worker_merge_files_into_table(:JobId, :TaskId, - ARRAY['textcolumn', 'binarycolumn'], - ARRAY['text', 'bytea']); - --- worker_execute_sql_task should only accept queries -select worker_execute_sql_task(0,0,'create table foo(a serial)',false); diff --git a/src/test/regress/sql/worker_remove_files.sql b/src/test/regress/sql/worker_remove_files.sql deleted file mode 100644 index 9b3ce85ba..000000000 --- a/src/test/regress/sql/worker_remove_files.sql +++ /dev/null @@ -1,5 +0,0 @@ --- Clear job directory used by previous tests - -\set JobId 201010 - -SELECT task_tracker_cleanup_job(:JobId); diff --git a/src/test/regress/worker_schedule b/src/test/regress/worker_schedule index 031bdff90..e702be378 100644 --- a/src/test/regress/worker_schedule +++ b/src/test/regress/worker_schedule @@ -18,17 +18,10 @@ test: worker_range_partition worker_range_partition_complex test: worker_hash_partition worker_hash_partition_complex test: worker_merge_range_files worker_merge_hash_files test: worker_binary_data_partition worker_null_data_partition -test: worker_check_invalid_arguments -test: worker_remove_files - -# ---------- -# All task tracker tests use the following tables -# ---------- -test: task_tracker_create_table -test: task_tracker_assign_task task_tracker_partition_task -test: task_tracker_cleanup_job # --------- # test that no tests leaked intermediate results. This should always be last # --------- -test: ensure_no_intermediate_data_leak +# there will be open some leak because we removed task tracker remove files +# this shouldn't be a problem in this schedule +# test: ensure_no_intermediate_data_leak