mirror of https://github.com/citusdata/citus.git
remove task-tracker in comments, documentation
parent
a3dc8fe2b5
commit
4308d867d9
|
@ -71,8 +71,6 @@
|
|||
* There are also a few limitations/trade-offs that are worth mentioning.
|
||||
* - The local execution on multiple shards might be slow because the execution
|
||||
* has to happen one task at a time (e.g., no parallelism).
|
||||
* - The local execution cannot be mixed with the executors other than adaptive,
|
||||
* namely task-tracker executor.
|
||||
* - Related with the previous item, COPY command cannot be mixed with local
|
||||
* execution in a transaction. The implication of that is any part of INSERT..SELECT
|
||||
* via coordinator cannot happen via the local execution.
|
||||
|
|
|
@ -3,8 +3,8 @@
|
|||
* multi_server_executor.c
|
||||
*
|
||||
* Function definitions for distributed task execution for adaptive
|
||||
* and task-tracker executors, and routines common to both. The common
|
||||
* routines are implement backend-side logic; and they trigger executions
|
||||
* executor.
|
||||
* routines are implemented backend-side logic; and they trigger executions
|
||||
* on the client-side via function hooks that they load.
|
||||
*
|
||||
* Copyright (c) Citus Data, Inc.
|
||||
|
|
|
@ -1357,8 +1357,7 @@ FinalizePlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan)
|
|||
|
||||
/*
|
||||
* FinalizeNonRouterPlan gets the distributed custom scan plan, and creates the
|
||||
* final master select plan on the top of this distributed plan for adaptive
|
||||
* and task-tracker executors.
|
||||
* final master select plan on the top of this distributed plan for adaptive executor.
|
||||
*/
|
||||
static PlannedStmt *
|
||||
FinalizeNonRouterPlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan,
|
||||
|
|
|
@ -4611,9 +4611,7 @@ MergeTaskList(MapMergeJob *mapMergeJob, List *mapTaskList, uint32 taskIdIndex)
|
|||
|
||||
/*
|
||||
* If replication factor is 1, then we know that we will use the first and
|
||||
* the only placement. If task tracker is used, then it will regenerate the
|
||||
* query string because if there are multiple placements then it does not
|
||||
* know in which placement the parent map task was successful.
|
||||
* the only placement.
|
||||
*/
|
||||
StringInfo mapFetchQueryString = makeStringInfo();
|
||||
appendStringInfo(mapFetchQueryString, MAP_OUTPUT_FETCH_COMMAND,
|
||||
|
|
|
@ -25,9 +25,6 @@
|
|||
* subqueries if the subquery can be executed on each shard by replacing
|
||||
* table names with shard names and concatenating the result.
|
||||
*
|
||||
* - Task-tracker queries that can be executed through a tree of
|
||||
* re-partitioning operations.
|
||||
*
|
||||
* These queries have very limited SQL support and only support basic
|
||||
* inner joins and subqueries without joins.
|
||||
*
|
||||
|
|
|
@ -2,10 +2,8 @@
|
|||
*
|
||||
* task_tracker_protocol.c
|
||||
*
|
||||
* The task tracker background process runs on every worker node. The following
|
||||
* routines allow for the master node to assign tasks to the task tracker, check
|
||||
* these tasks' statuses, and remove these tasks when they are no longer needed.
|
||||
*
|
||||
* The methods in the file are deprecated.
|
||||
*
|
||||
* Copyright (c) Citus Data, Inc.
|
||||
*
|
||||
* $Id$
|
||||
|
@ -25,11 +23,7 @@ PG_FUNCTION_INFO_V1(task_tracker_cleanup_job);
|
|||
PG_FUNCTION_INFO_V1(task_tracker_conninfo_cache_invalidate);
|
||||
|
||||
|
||||
/*
|
||||
* task_tracker_assign_task creates a new task in the shared hash or updates an
|
||||
* already existing task. The function also creates a schema for the job if it
|
||||
* doesn't already exist.
|
||||
*/
|
||||
/* This UDF is deprecated.*/
|
||||
Datum
|
||||
task_tracker_assign_task(PG_FUNCTION_ARGS)
|
||||
{
|
||||
|
@ -39,7 +33,7 @@ task_tracker_assign_task(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
|
||||
/* Returns the task status of an already existing task. */
|
||||
/* This UDF is deprecated.*/
|
||||
Datum
|
||||
task_tracker_task_status(PG_FUNCTION_ARGS)
|
||||
{
|
||||
|
@ -49,10 +43,7 @@ task_tracker_task_status(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* task_tracker_cleanup_job finds all tasks for the given job, and cleans up
|
||||
* files, connections, and shared hash enties associated with these tasks.
|
||||
*/
|
||||
/* This UDF is deprecated.*/
|
||||
Datum
|
||||
task_tracker_cleanup_job(PG_FUNCTION_ARGS)
|
||||
{
|
||||
|
@ -62,13 +53,7 @@ task_tracker_cleanup_job(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* task_tracker_conninfo_cache_invalidate is a trigger function that signals to
|
||||
* the task tracker to refresh its conn params cache after an authinfo change.
|
||||
*
|
||||
* NB: We decided there is little point in checking permissions here, there
|
||||
* are much easier ways to waste CPU than causing cache invalidations.
|
||||
*/
|
||||
/* This UDF is deprecated.*/
|
||||
Datum
|
||||
task_tracker_conninfo_cache_invalidate(PG_FUNCTION_ARGS)
|
||||
{
|
||||
|
|
|
@ -163,7 +163,7 @@ worker_repartition_cleanup(PG_FUNCTION_ARGS)
|
|||
|
||||
/*
|
||||
* worker_merge_files_into_table creates a task table within the job's schema,
|
||||
* which should have already been created by the task tracker protocol, and
|
||||
* which should have already been created by repartition join execution, and
|
||||
* copies files in its task directory into this table. If the schema doesn't
|
||||
* exist, the function defaults to the 'public' schema. Note that, unlike
|
||||
* partitioning functions, this function is not always idempotent. On success,
|
||||
|
@ -200,8 +200,8 @@ worker_merge_files_into_table(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
/*
|
||||
* If the schema for the job isn't already created by the task tracker
|
||||
* protocol, we fall to using the default 'public' schema.
|
||||
* If the schema for the job isn't already created by the repartition join
|
||||
* execution, we fall to using the default 'public' schema.
|
||||
*/
|
||||
bool schemaExists = JobSchemaExists(jobSchemaName);
|
||||
if (!schemaExists)
|
||||
|
@ -245,18 +245,7 @@ worker_merge_files_into_table(PG_FUNCTION_ARGS)
|
|||
}
|
||||
|
||||
|
||||
/*
|
||||
* worker_merge_files_and_run_query creates a merge task table within the job's
|
||||
* schema, which should have already been created by the task tracker protocol.
|
||||
* It copies files in its task directory into this table. Then it runs final
|
||||
* query to create result table of the job.
|
||||
*
|
||||
* Note that here we followed a different approach to create a task table for merge
|
||||
* files than worker_merge_files_into_table(). In future we should unify these
|
||||
* two approaches. For this purpose creating a directory_fdw extension and using
|
||||
* it would make sense. Then we can merge files with a query or without query
|
||||
* through directory_fdw.
|
||||
*/
|
||||
/* This UDF is deprecated.*/
|
||||
Datum
|
||||
worker_merge_files_and_run_query(PG_FUNCTION_ARGS)
|
||||
{
|
||||
|
|
|
@ -372,10 +372,6 @@ ArrayObjectCount(ArrayType *arrayObject)
|
|||
StringInfo
|
||||
InitTaskDirectory(uint64 jobId, uint32 taskId)
|
||||
{
|
||||
/*
|
||||
* If the task tracker assigned this task (regular case), the tracker should
|
||||
* have already created the job directory.
|
||||
*/
|
||||
StringInfo jobDirectoryName = JobDirectoryName(jobId);
|
||||
StringInfo taskDirectoryName = TaskDirectoryName(jobId, taskId);
|
||||
|
||||
|
|
|
@ -2,7 +2,7 @@
|
|||
*
|
||||
* worker_sql_task_protocol.c
|
||||
*
|
||||
* Routines for executing SQL tasks during task-tracker execution.
|
||||
* Routines for executing SQL tasks.
|
||||
*
|
||||
* Copyright (c) Citus Data, Inc.
|
||||
*
|
||||
|
|
|
@ -42,7 +42,7 @@ output_files := $(patsubst $(citus_abs_srcdir)/output/%.source,expected/%.out, $
|
|||
# intermediate, for muscle memory backward compatibility.
|
||||
check: check-full
|
||||
# check-full triggers all tests that ought to be run routinely
|
||||
check-full: check-multi check-multi-mx check-multi-task-tracker-extra check-worker check-follower-cluster check-failure
|
||||
check-full: check-multi check-multi-mx check-worker check-follower-cluster check-failure
|
||||
|
||||
|
||||
ISOLATION_DEPDIR=.deps/isolation
|
||||
|
|
|
@ -1401,7 +1401,7 @@ ROLLBACK;
|
|||
BEGIN;
|
||||
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)
|
||||
SET LOCAL citus.task_executor_type = 'task-tracker';
|
||||
|
||||
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
|
||||
|
@ -1414,7 +1414,7 @@ ROLLBACK;
|
|||
-- local execution should not be executed locally
|
||||
-- becase a task-tracker query has already been executed
|
||||
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
|
||||
|
|
|
@ -145,7 +145,6 @@ SELECT * FROM raw_table WHERE ID = 1;
|
|||
1 | 20
|
||||
(2 rows)
|
||||
|
||||
-- Task tracker query test
|
||||
CREATE TABLE tt1
|
||||
(
|
||||
id int,
|
||||
|
|
|
@ -5,7 +5,7 @@ SET citus.next_shard_id TO 650000;
|
|||
-- Set configuration to print table join order and pruned shards
|
||||
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.task_executor_type = 'adaptive'; -- 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;
|
||||
|
|
|
@ -5,7 +5,7 @@ SET citus.next_shard_id TO 660000;
|
|||
-- Enable configuration to print table join order
|
||||
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.task_executor_type = 'adaptive'; -- 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
|
||||
|
|
|
@ -152,7 +152,7 @@ INSERT INTO repartition_udt_other values (10, '(2,1)'::test_udt, 'foo');
|
|||
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.task_executor_type = 'adaptive';
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
-- Query that should result in a repartition
|
||||
-- join on int column, and be empty.
|
||||
|
|
|
@ -3,8 +3,7 @@
|
|||
--
|
||||
-- Tests that cover repartition join planning. Note that we explicitly start a
|
||||
-- transaction block here so that we don't emit debug messages with changing
|
||||
-- transaction ids in them. Also, we set the executor type to task tracker
|
||||
-- executor here, as we cannot run repartition jobs with real time executor.
|
||||
-- transaction ids in them.
|
||||
SET citus.next_shard_id TO 690000;
|
||||
SET citus.enable_unique_job_ids TO off;
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
|
|
|
@ -1,9 +1,7 @@
|
|||
--
|
||||
-- MULTI_REPARTITION_JOIN_PRUNING
|
||||
--
|
||||
-- Tests covering partition and join-pruning for repartition joins. Note that we
|
||||
-- set executor type to task tracker executor here, as we cannot run repartition
|
||||
-- jobs with real time executor.
|
||||
-- Tests covering partition and join-pruning for repartition joins.
|
||||
SET citus.next_shard_id TO 700000;
|
||||
SET client_min_messages TO DEBUG2;
|
||||
SET citus.enable_repartition_joins to ON;
|
||||
|
|
|
@ -164,7 +164,7 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other
|
|||
(0 rows)
|
||||
|
||||
-- Query that should result in a repartition join on UDT column.
|
||||
SET citus.task_executor_type = 'task-tracker';
|
||||
SET citus.task_executor_type = 'adaptive';
|
||||
SET citus.log_multi_join_order = true;
|
||||
EXPLAIN (COSTS OFF)
|
||||
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||
|
|
|
@ -215,12 +215,12 @@ SELECT * FROM articles, position('om' in 'Thomas') ORDER BY 2 DESC, 1 DESC, 3 DE
|
|||
|
||||
-- subqueries are supported in WHERE clause in Citus even if the relations are not distributed
|
||||
SELECT * FROM articles WHERE author_id IN (SELECT id FROM authors WHERE name LIKE '%a');
|
||||
ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker'
|
||||
ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'adaptive'
|
||||
-- subqueries are supported in FROM clause
|
||||
SELECT articles.id,test.word_count
|
||||
FROM articles, (SELECT id, word_count FROM articles) AS test WHERE test.id = articles.id
|
||||
ORDER BY articles.id;
|
||||
ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'task-tracker'
|
||||
ERROR: Complex subqueries and CTEs are not supported when task_executor_type is set to 'adaptive'
|
||||
-- subqueries are not supported in SELECT clause
|
||||
SELECT a.title AS name, (SELECT a2.id FROM articles_single_shard a2 WHERE a.id = a2.id LIMIT 1)
|
||||
AS special_price FROM articles a;
|
||||
|
|
|
@ -519,7 +519,7 @@ SELECT create_distributed_table('dist_1', 'a');
|
|||
(1 row)
|
||||
|
||||
INSERT INTO dist_1 SELECT x,10-x FROM generate_series(1,10) x;
|
||||
SET citus.task_executor_type to 'task-tracker';
|
||||
SET citus.task_executor_type to 'adaptive';
|
||||
SELECT COUNT(*) FROM dist_1 f, dist_1 s WHERE f.a = s.b;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -266,8 +266,6 @@ SELECT count(*) FROM test WHERE false GROUP BY GROUPING SETS (x,y);
|
|||
(0 rows)
|
||||
|
||||
RESET citus.task_assignment_policy;
|
||||
-- single node task tracker tests:
|
||||
SET citus.task_executor_type to 'task-tracker';
|
||||
SELECT count(*) FROM test;
|
||||
count
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -375,7 +375,7 @@ SELECT count(*) FROM cte, users_table where cte.count=user_id;
|
|||
0
|
||||
(1 row)
|
||||
|
||||
SET citus.task_executor_type='task-tracker';
|
||||
|
||||
-- CTEs shouldn't be able to terminate a task-tracker query
|
||||
WITH cte_1 AS (
|
||||
SELECT
|
||||
|
|
|
@ -16,7 +16,6 @@ SELECT cte.user_id, cte.value_2 FROM cte,cte2 ORDER BY 1,2 LIMIT 10;
|
|||
|
||||
|
||||
SET citus.max_intermediate_result_size TO 9;
|
||||
-- regular task-tracker CTE should fail
|
||||
WITH cte AS
|
||||
(
|
||||
SELECT
|
||||
|
@ -172,7 +171,7 @@ cte4 AS (
|
|||
SELECT * FROM cte UNION ALL
|
||||
SELECT * FROM cte4 ORDER BY 1,2,3,4,5 LIMIT 5;
|
||||
|
||||
-- regular task-tracker CTE, should work since -1 disables the limit
|
||||
-- regular adaptive executor CTE, should work since -1 disables the limit
|
||||
WITH cte AS
|
||||
(
|
||||
SELECT
|
||||
|
|
|
@ -695,19 +695,13 @@ BEGIN;
|
|||
ROLLBACK;
|
||||
|
||||
|
||||
-- task-tracker select execution
|
||||
BEGIN;
|
||||
DELETE FROM distributed_table WHERE key = 500;
|
||||
|
||||
SET LOCAL citus.task_executor_type = 'task-tracker';
|
||||
|
||||
SELECT count(*) FROM distributed_table;
|
||||
ROLLBACK;
|
||||
|
||||
-- local execution should not be executed locally
|
||||
-- becase a task-tracker query has already been executed
|
||||
BEGIN;
|
||||
SET LOCAL citus.task_executor_type = 'task-tracker';
|
||||
SET LOCAL client_min_messages TO INFO;
|
||||
SELECT count(*) FROM distributed_table;
|
||||
SET LOCAL client_min_messages TO LOG;
|
||||
|
|
|
@ -90,7 +90,6 @@ SET citus.multi_task_query_log_level TO notice;
|
|||
-- Shouldn't log since it is a router select query
|
||||
SELECT * FROM raw_table WHERE ID = 1;
|
||||
|
||||
-- Task tracker query test
|
||||
CREATE TABLE tt1
|
||||
(
|
||||
id int,
|
||||
|
@ -113,7 +112,6 @@ 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
|
||||
SELECT tt1.id, tt2.count from tt1,tt2 where tt1.id = tt2.id ORDER BY 1;
|
||||
|
||||
SET citus.task_executor_type to DEFAULT;
|
||||
|
|
|
@ -9,7 +9,6 @@ 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;
|
||||
|
|
|
@ -10,7 +10,6 @@ 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;
|
||||
|
||||
|
|
|
@ -182,7 +182,6 @@ 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
|
||||
|
|
|
@ -641,15 +641,6 @@ SET client_min_messages to 'DEBUG2';
|
|||
CREATE MATERIALIZED VIEW mv_articles_hash_mx_error AS
|
||||
SELECT * FROM articles_hash_mx WHERE author_id in (1,2);
|
||||
|
||||
-- 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
|
||||
----SELECT id
|
||||
-- FROM articles_hash_mx
|
||||
-- WHERE author_id = 1;
|
||||
|
||||
-- insert query is router plannable even under task-tracker
|
||||
INSERT INTO articles_hash_mx VALUES (51, 1, 'amateus', 1814);
|
||||
|
||||
-- verify insert is successfull (not router plannable and executable)
|
||||
|
|
|
@ -863,7 +863,6 @@ SELECT * FROM partitioning_locks 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;
|
||||
|
||||
-- test locks on task-tracker SELECT
|
||||
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;
|
||||
|
|
|
@ -362,8 +362,6 @@ SELECT real_time_partition_column_select(4);
|
|||
SELECT real_time_partition_column_select(5);
|
||||
SELECT real_time_partition_column_select(6);
|
||||
|
||||
-- check task-tracker executor
|
||||
|
||||
CREATE FUNCTION task_tracker_non_partition_column_select(value_arg int)
|
||||
RETURNS TABLE(key int, value int) AS $$
|
||||
DECLARE
|
||||
|
|
|
@ -393,8 +393,6 @@ EXECUTE prepared_real_time_partition_column_select(4);
|
|||
EXECUTE prepared_real_time_partition_column_select(5);
|
||||
EXECUTE prepared_real_time_partition_column_select(6);
|
||||
|
||||
-- check task-tracker executor
|
||||
|
||||
PREPARE prepared_task_tracker_non_partition_column_select(int) AS
|
||||
SELECT
|
||||
prepare_table.key,
|
||||
|
|
|
@ -3,8 +3,7 @@
|
|||
--
|
||||
-- Tests that cover repartition join planning. Note that we explicitly start a
|
||||
-- transaction block here so that we don't emit debug messages with changing
|
||||
-- transaction ids in them. Also, we set the executor type to task tracker
|
||||
-- executor here, as we cannot run repartition jobs with real time executor.
|
||||
-- transaction ids in them.
|
||||
|
||||
|
||||
SET citus.next_shard_id TO 690000;
|
||||
|
|
|
@ -1,9 +1,7 @@
|
|||
--
|
||||
-- MULTI_REPARTITION_JOIN_PRUNING
|
||||
--
|
||||
-- Tests covering partition and join-pruning for repartition joins. Note that we
|
||||
-- set executor type to task tracker executor here, as we cannot run repartition
|
||||
-- jobs with real time executor.
|
||||
-- Tests covering partition and join-pruning for repartition joins.
|
||||
|
||||
|
||||
SET citus.next_shard_id TO 700000;
|
||||
|
|
|
@ -190,7 +190,6 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other
|
|||
WHERE repartition_udt.pk = 1;
|
||||
|
||||
-- Query that should result in a repartition join on UDT column.
|
||||
SET citus.task_executor_type = 'task-tracker';
|
||||
SET citus.log_multi_join_order = true;
|
||||
|
||||
EXPLAIN (COSTS OFF)
|
||||
|
|
|
@ -1113,13 +1113,11 @@ CREATE MATERIALIZED VIEW mv_articles_hash_data AS
|
|||
SELECT * FROM articles_hash WHERE author_id in (1,2);
|
||||
SELECT * FROM mv_articles_hash_data ORDER BY 1, 2, 3, 4;
|
||||
|
||||
-- router planner/executor is now enabled for task-tracker executor
|
||||
SELECT id
|
||||
FROM articles_hash
|
||||
WHERE author_id = 1
|
||||
ORDER BY 1;
|
||||
|
||||
-- insert query is router plannable even under task-tracker
|
||||
INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814), (52, 1, 'second amateus', 2824);
|
||||
|
||||
-- verify insert is successful (not router plannable and executable)
|
||||
|
|
|
@ -805,12 +805,10 @@ CREATE MATERIALIZED VIEW mv_articles_hash_empty AS
|
|||
SELECT * FROM mv_articles_hash_empty;
|
||||
|
||||
|
||||
-- fast-path router planner/executor is enabled for task-tracker executor
|
||||
SELECT id
|
||||
FROM articles_hash
|
||||
WHERE author_id = 1;
|
||||
|
||||
-- insert query is router plannable even under task-tracker
|
||||
INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814), (52, 1, 'second amateus', 2824);
|
||||
|
||||
-- verify insert is successfull (not router plannable and executable)
|
||||
|
|
|
@ -151,7 +151,6 @@ CREATE TABLE dist_1 (a int , b int);
|
|||
SELECT create_distributed_table('dist_1', 'a');
|
||||
INSERT INTO dist_1 SELECT x,10-x FROM generate_series(1,10) x;
|
||||
|
||||
SET citus.task_executor_type to 'task-tracker';
|
||||
SELECT COUNT(*) FROM dist_1 f, dist_1 s WHERE f.a = s.b;
|
||||
|
||||
SET client_min_messages TO ERROR;
|
||||
|
|
|
@ -115,8 +115,6 @@ SELECT count(*) FROM test WHERE false;
|
|||
SELECT count(*) FROM test WHERE false GROUP BY GROUPING SETS (x,y);
|
||||
RESET citus.task_assignment_policy;
|
||||
|
||||
-- single node task tracker tests:
|
||||
SET citus.task_executor_type to 'task-tracker';
|
||||
SELECT count(*) FROM test;
|
||||
|
||||
-- INSERT SELECT from distributed table to local table
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
-- Confirm we can use local, router, real-time, and task-tracker execution
|
||||
-- Confirm we can use local, and adaptive execution
|
||||
|
||||
CREATE SCHEMA with_executors;
|
||||
SET search_path TO with_executors, public;
|
||||
|
@ -105,7 +105,7 @@ WITH cte AS (
|
|||
SELECT * FROM cte WHERE uid=1 ORDER BY 2 LIMIT 5;
|
||||
|
||||
|
||||
-- CTEs should be able to use task-tracker queries
|
||||
-- CTEs should be able to use adaptive executor
|
||||
WITH cte AS (
|
||||
WITH task_tracker_1 AS (
|
||||
SELECT
|
||||
|
@ -294,8 +294,6 @@ WITH cte AS (
|
|||
SELECT count(*) FROM cte, users_table where cte.count=user_id;
|
||||
|
||||
|
||||
SET citus.task_executor_type='task-tracker';
|
||||
-- CTEs shouldn't be able to terminate a task-tracker query
|
||||
WITH cte_1 AS (
|
||||
SELECT
|
||||
u_table.user_id as u_id, e_table.event_type
|
||||
|
|
Loading…
Reference in New Issue