mirror of https://github.com/citusdata/citus.git
Remove task tracker executor (#3850)
* use adaptive executor even if task-tracker is set
* Update check-multi-mx tests for adaptive executor
Basically repartition joins are enabled where necessary. For parallel
tests max adaptive executor pool size is decresed to 2, otherwise we
would get too many clients error.
* Update limit_intermediate_size test
It seems that when we use adaptive executor instead of task tracker, we
exceed the intermediate result size less in the test. Therefore updated
the tests accordingly.
* Update multi_router_planner
It seems that there is one problem with multi_router_planner when we use
adaptive executor, we should fix the following error:
+ERROR: relation "authors_range_840010" does not exist
+CONTEXT: while executing command on localhost:57637
* update repartition join tests for check-multi
* update isolation tests for repartitioning
* Error out if shard_replication_factor > 1 with repartitioning
As we are removing the task tracker, we cannot switch to it if
shard_replication_factor > 1. In that case, we simply error out.
* Remove MULTI_EXECUTOR_TASK_TRACKER
* Remove multi_task_tracker_executor
Some utility methods are moved to task_execution_utils.c.
* Remove task tracker protocol methods
* Remove task_tracker.c methods
* remove unused methods from multi_server_executor
* fix style
* remove task tracker specific tests from worker_schedule
* comment out task tracker udf calls in tests
We were using task tracker udfs to test permissions in
multi_multiuser.sql. We should find some other way to test them, then we
should remove the commented out task tracker calls.
* remove task tracker test from follower schedule
* remove task tracker tests from multi mx schedule
* Remove task-tracker specific functions from worker functions
* remove multi task tracker extra schedule
* Remove unused methods from multi physical planner
* remove task_executor_type related things in tests
* remove LoadTuplesIntoTupleStore
* Do initial cleanup for repartition leftovers
During startup, task tracker would call TrackerCleanupJobDirectories and
TrackerCleanupJobSchemas to clean up leftover directories and job
schemas. With adaptive executor, while doing repartitions it is possible
to leak these things as well. We don't retry cleanups, so it is possible
to have leftover in case of errors.
TrackerCleanupJobDirectories is renamed as
RepartitionCleanupJobDirectories since it is repartition specific now,
however TrackerCleanupJobSchemas cannot be used currently because it is
task tracker specific. The thing is that this function is a no-op
currently.
We should add cleaning up intermediate schemas to DoInitialCleanup
method when that problem is solved(We might want to solve it in this PR
as well)
* Revert "remove task tracker tests from multi mx schedule"
This reverts commit 03ecc0a681
.
* update multi mx repartition parallel tests
* not error with task_tracker_conninfo_cache_invalidate
* not run 4 repartition queries in parallel
It seems that when we run 4 repartition queries in parallel we get too
many clients error on CI even though we don't get it locally. Our guess
is that, it is because we open/close many connections without doing some
work and postgres has some delay to close the connections. Hence even
though connections are removed from the pg_stat_activity, they might
still not be closed. If the above assumption is correct, it is unlikely
for it to happen in practice because:
- There is some network latency in clusters, so this leaves some times
for connections to be able to close
- Repartition joins return some data and that also leaves some time for
connections to be fully closed.
As we don't get this error in our local, we currently assume that it is
not a bug. Ideally this wouldn't happen when we get rid of the
task-tracker repartition methods because they don't do any pruning and
might be opening more connections than necessary.
If this still gives us "too many clients" error, we can try to increase
the max_connections in our test suite(which is 100 by default).
Also there are different places where this error is given in postgres,
but adding some backtrace it seems that we get this from
ProcessStartupPacket. The backtraces can be found in this link:
https://circleci.com/gh/citusdata/citus/138702
* Set distributePlan->relationIdList when it is needed
It seems that we were setting the distributedPlan->relationIdList after
JobExecutorType is called, which would choose task-tracker if
replication factor > 1 and there is a repartition query. However, it
uses relationIdList to decide if the query has a repartition query, and
since it was not set yet, it would always think it is not a repartition
query and would choose adaptive executor when it should choose
task-tracker.
* use adaptive executor even with shard_replication_factor > 1
It seems that we were already using adaptive executor when
replication_factor > 1. So this commit removes the check.
* remove multi_resowner.c and deprecate some settings
* remove TaskExecution related leftovers
* change deprecated API error message
* not recursively plan single relatition repartition subquery
* recursively plan single relation repartition subquery
* test depreceated task tracker functions
* fix overlapping shard intervals in range-distributed test
* fix error message for citus_metadata_container
* drop task-tracker deprecated functions
* put the implemantation back to worker_cleanup_job_schema_cachesince citus cloud uses it
* drop some functions, add downgrade script
Some deprecated functions are dropped.
Downgrade script is added.
Some gucs are deprecated.
A new guc for repartition joins bucket size is added.
* order by a test to fix flappiness
pull/4041/head
parent
339d43357c
commit
b3af63c8ce
|
@ -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]
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -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";
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -521,17 +521,8 @@ SubqueryMultiNodeTree(Query *originalQuery, Query *queryTree,
|
|||
multiQueryNode = SubqueryPushdownMultiNodeTree(originalQuery);
|
||||
}
|
||||
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);
|
||||
}
|
||||
|
||||
List *subqueryEntryList = SubqueryEntryList(queryTree);
|
||||
RangeTblEntry *subqueryRangeTableEntry = (RangeTblEntry *) linitial(
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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),
|
||||
|
||||
|
|
|
@ -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)
|
||||
{
|
||||
|
|
|
@ -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);
|
||||
}
|
|
@ -0,0 +1,224 @@
|
|||
|
||||
#include "postgres.h"
|
||||
#include "miscadmin.h"
|
||||
|
||||
#include <sys/stat.h>
|
||||
#include <unistd.h>
|
||||
#include <math.h>
|
||||
|
||||
#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;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -17,35 +17,6 @@
|
|||
#include "funcapi.h"
|
||||
#include "miscadmin.h"
|
||||
|
||||
#include <time.h>
|
||||
|
||||
#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")));
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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"
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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 */
|
||||
|
|
|
@ -60,7 +60,6 @@ typedef enum CitusNodeTag
|
|||
T_UsedDistributedSubPlan,
|
||||
T_Task,
|
||||
T_LocalPlannedStatement,
|
||||
T_TaskExecution,
|
||||
T_ShardInterval,
|
||||
T_ShardPlacement,
|
||||
T_RelationShard,
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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 */
|
||||
|
||||
/*
|
||||
|
|
|
@ -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 */
|
|
@ -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 */
|
||||
|
|
|
@ -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 */
|
|
@ -0,0 +1,6 @@
|
|||
#ifndef TASK_EXECUTION_UTILS_H
|
||||
#define TASK_EXECUTION_UTILS_H
|
||||
|
||||
extern List * CreateTaskListForJobTree(List *jobTaskList);
|
||||
|
||||
#endif /* TASK_EXECUTION_UTILS_H */
|
|
@ -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 */
|
|
@ -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 */
|
|
@ -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 */
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -60,7 +60,7 @@ 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";
|
||||
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,7 +399,7 @@ 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";
|
||||
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
|
||||
|
|
|
@ -60,7 +60,7 @@ 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";
|
||||
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
|
||||
|
@ -492,7 +492,7 @@ 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";
|
||||
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
|
||||
|
|
|
@ -60,7 +60,7 @@ 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";
|
||||
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,7 +335,7 @@ 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";
|
||||
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
|
||||
|
|
|
@ -60,7 +60,7 @@ 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";
|
||||
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,7 +413,7 @@ 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";
|
||||
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
|
||||
|
|
|
@ -60,7 +60,6 @@ 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;
|
||||
|
||||
id data int_data id data int_data
|
||||
|
@ -382,7 +381,6 @@ 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;
|
||||
|
||||
id data int_data id data int_data
|
||||
|
|
|
@ -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;
|
||||
<waiting ...>
|
||||
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;
|
||||
<waiting ...>
|
||||
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;
|
||||
<waiting ...>
|
||||
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;
|
||||
<waiting ...>
|
||||
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;
|
||||
<waiting ...>
|
||||
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;
|
||||
<waiting ...>
|
||||
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
|
||||
|
|
|
@ -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 (
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
---------------------------------------------------------------------
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
|||
<Plan>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>Citus Task-Tracker</Custom-Plan-Provider>
|
||||
<Custom-Plan-Provider>Citus Adaptive</Custom-Plan-Provider>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Distributed-Query>
|
||||
<Job>
|
||||
|
@ -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)
|
||||
|
|
|
@ -420,7 +420,13 @@ ALTER EXTENSION citus UPDATE TO '9.5-1';
|
|||
SELECT * FROM print_extension_changes();
|
||||
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
|
||||
|
|
|
@ -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;
|
|
@ -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);
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
|||
<Plan>
|
||||
<Node-Type>Custom Scan</Node-Type>
|
||||
<Parent-Relationship>Outer</Parent-Relationship>
|
||||
<Custom-Plan-Provider>Citus Task-Tracker</Custom-Plan-Provider>
|
||||
<Custom-Plan-Provider>Citus Adaptive</Custom-Plan-Provider>
|
||||
<Parallel-Aware>false</Parallel-Aware>
|
||||
<Distributed-Query>
|
||||
<Job>
|
||||
|
@ -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:
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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');
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
@ -321,7 +321,7 @@ FROM
|
|||
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: All
|
||||
(3 rows)
|
||||
|
@ -336,7 +336,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: All
|
||||
(3 rows)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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";
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue