mirror of https://github.com/citusdata/citus.git
Refactor master query to be planned by postgres' planner (#3326)
DESCRIPTION: Replace the query planner for the coordinator part with the postgres planner Closes #2761 Citus had a simple rule based planner for the query executed on the query coordinator. This planner grew over time with the addigion of SQL support till it was getting close to the functionality of the postgres planner. Except the code was brittle and its complexity rose which made it hard to add new SQL support. Given its resemblance with the postgres planner it was a long outstanding wish to replace our hand crafted planner with the well supported postgres planner. This patch replaces our planner with a call to postgres' planner. Due to the functionality of the postgres planner we needed to support both projections and filters/quals on the citus custom scan node. When a sort operation is planned above the custom scan it might require fields to be reordered in the custom scan before returning the tuple (projection). The postgres planner assumes every custom scan node implements projections. Because we controlled the plan that was created we prevented reordering in the custom scan and never had implemented it before. A same optimisation applies to having clauses that could have been where clauses. Instead of applying the filter as a having on the aggregate it will push it down into the plan which could reach a custom scan node. For both filters and projections we have implemented them when tuples are read from the tuple store. If no projections or filters are required it will directly return the tuple from the tuple store. Otherwise it will loop tuples from the tuple store through the filter and projection until a tuple is found and returned. Besides filters being pushed down a side effect of having quals that could have been a where clause is that a call to read intermediate result could be called before the first tuple is fetched from the custom scan. This failed because the intermediate result would only be pulled to the coordinator on the first tuple fetch. To overcome this problem we do run the distributed subplans now before we run the postgres executor. This ensures the intermediate result is present on the coordinator in time. We do account for total time instrumentation by removing the instrumentation before handing control to the psotgres executor and update the timings our self. For future SQL support it is enough to create a valid query structure for the part of the query to be executed on the query coordinating node. As a utility we do serialise and print the query at debug level4 for engineers to inspect what kind of query is being planned on the query coordinator.pull/3539/head
parent
0c4f9e230d
commit
a77ed9cd23
|
@ -624,6 +624,28 @@ static void ProcessWaitEvents(DistributedExecution *execution, WaitEvent *events
|
||||||
eventCount, bool *cancellationReceived);
|
eventCount, bool *cancellationReceived);
|
||||||
static long MillisecondsBetweenTimestamps(instr_time startTime, instr_time endTime);
|
static long MillisecondsBetweenTimestamps(instr_time startTime, instr_time endTime);
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* AdaptiveExecutorPreExecutorRun gets called right before postgres starts its executor
|
||||||
|
* run. Given that the result of our subplans would be evaluated before the first call to
|
||||||
|
* the exec function of our custom scan we make sure our subplans have executed before.
|
||||||
|
*/
|
||||||
|
void
|
||||||
|
AdaptiveExecutorPreExecutorRun(CitusScanState *scanState)
|
||||||
|
{
|
||||||
|
DistributedPlan *distributedPlan = scanState->distributedPlan;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* PostgreSQL takes locks on all partitions in the executor. It's not entirely
|
||||||
|
* clear why this is necessary (instead of locking the parent during DDL), but
|
||||||
|
* we do the same for consistency.
|
||||||
|
*/
|
||||||
|
LockPartitionsForDistributedPlan(distributedPlan);
|
||||||
|
|
||||||
|
ExecuteSubPlans(distributedPlan);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* AdaptiveExecutor is called via CitusExecScan on the
|
* AdaptiveExecutor is called via CitusExecScan on the
|
||||||
* first call of CitusExecScan. The function fills the tupleStore
|
* first call of CitusExecScan. The function fills the tupleStore
|
||||||
|
@ -649,15 +671,6 @@ AdaptiveExecutor(CitusScanState *scanState)
|
||||||
/* we should only call this once before the scan finished */
|
/* we should only call this once before the scan finished */
|
||||||
Assert(!scanState->finishedRemoteScan);
|
Assert(!scanState->finishedRemoteScan);
|
||||||
|
|
||||||
/*
|
|
||||||
* PostgreSQL takes locks on all partitions in the executor. It's not entirely
|
|
||||||
* clear why this is necessary (instead of locking the parent during DDL), but
|
|
||||||
* we do the same for consistency.
|
|
||||||
*/
|
|
||||||
LockPartitionsForDistributedPlan(distributedPlan);
|
|
||||||
|
|
||||||
ExecuteSubPlans(distributedPlan);
|
|
||||||
|
|
||||||
bool hasDependentJobs = HasDependentJobs(job);
|
bool hasDependentJobs = HasDependentJobs(job);
|
||||||
if (hasDependentJobs)
|
if (hasDependentJobs)
|
||||||
{
|
{
|
||||||
|
|
|
@ -49,6 +49,7 @@ static Node * DelayedErrorCreateScan(CustomScan *scan);
|
||||||
static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags);
|
static void CitusBeginScan(CustomScanState *node, EState *estate, int eflags);
|
||||||
static void CitusBeginScanWithCoordinatorProcessing(CustomScanState *node, EState *estate,
|
static void CitusBeginScanWithCoordinatorProcessing(CustomScanState *node, EState *estate,
|
||||||
int eflags);
|
int eflags);
|
||||||
|
static void CitusPreExecScan(CitusScanState *scanState);
|
||||||
static void HandleDeferredShardPruningForFastPathQueries(
|
static void HandleDeferredShardPruningForFastPathQueries(
|
||||||
DistributedPlan *distributedPlan);
|
DistributedPlan *distributedPlan);
|
||||||
static void HandleDeferredShardPruningForInserts(DistributedPlan *distributedPlan);
|
static void HandleDeferredShardPruningForInserts(DistributedPlan *distributedPlan);
|
||||||
|
@ -114,6 +115,29 @@ static CustomExecMethods CoordinatorInsertSelectCustomExecMethods = {
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* IsCitusCustomState returns if a given PlanState node is a CitusCustomState node.
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
IsCitusCustomState(PlanState *planState)
|
||||||
|
{
|
||||||
|
if (!IsA(planState, CustomScanState))
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
CustomScanState *css = castNode(CustomScanState, planState);
|
||||||
|
if (css->methods == &AdaptiveExecutorCustomExecMethods ||
|
||||||
|
css->methods == &TaskTrackerCustomExecMethods ||
|
||||||
|
css->methods == &CoordinatorInsertSelectCustomExecMethods)
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* Let PostgreSQL know about Citus' custom scan nodes.
|
* Let PostgreSQL know about Citus' custom scan nodes.
|
||||||
*/
|
*/
|
||||||
|
@ -141,7 +165,24 @@ CitusBeginScan(CustomScanState *node, EState *estate, int eflags)
|
||||||
CitusScanState *scanState = (CitusScanState *) node;
|
CitusScanState *scanState = (CitusScanState *) node;
|
||||||
|
|
||||||
#if PG_VERSION_NUM >= 120000
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Since we are using a tuplestore we cannot use the virtual tuples postgres had
|
||||||
|
* already setup on the CustomScan. Instead we need to reinitialize the tuples as
|
||||||
|
* minimal.
|
||||||
|
*
|
||||||
|
* During initialization postgres also created the projection information and the
|
||||||
|
* quals, but both are 'compiled' to be executed on virtual tuples. Since we replaced
|
||||||
|
* the tuples with minimal tuples we also compile both the projection and the quals
|
||||||
|
* on to these 'new' tuples.
|
||||||
|
*/
|
||||||
ExecInitResultSlot(&scanState->customScanState.ss.ps, &TTSOpsMinimalTuple);
|
ExecInitResultSlot(&scanState->customScanState.ss.ps, &TTSOpsMinimalTuple);
|
||||||
|
|
||||||
|
ExecInitScanTupleSlot(node->ss.ps.state, &node->ss, node->ss.ps.scandesc,
|
||||||
|
&TTSOpsMinimalTuple);
|
||||||
|
ExecAssignScanProjectionInfoWithVarno(&node->ss, INDEX_VAR);
|
||||||
|
|
||||||
|
node->ss.ps.qual = ExecInitQual(node->ss.ps.plan->qual, (PlanState *) node);
|
||||||
#endif
|
#endif
|
||||||
|
|
||||||
DistributedPlan *distributedPlan = scanState->distributedPlan;
|
DistributedPlan *distributedPlan = scanState->distributedPlan;
|
||||||
|
@ -158,6 +199,16 @@ CitusBeginScan(CustomScanState *node, EState *estate, int eflags)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CitusPreExecScan is called right before postgres' executor starts pulling tuples.
|
||||||
|
*/
|
||||||
|
static void
|
||||||
|
CitusPreExecScan(CitusScanState *scanState)
|
||||||
|
{
|
||||||
|
AdaptiveExecutorPreExecutorRun(scanState);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* CitusExecScan is called when a tuple is pulled from a custom scan.
|
* CitusExecScan is called when a tuple is pulled from a custom scan.
|
||||||
* On the first call, it executes the distributed query and writes the
|
* On the first call, it executes the distributed query and writes the
|
||||||
|
@ -176,9 +227,7 @@ CitusExecScan(CustomScanState *node)
|
||||||
scanState->finishedRemoteScan = true;
|
scanState->finishedRemoteScan = true;
|
||||||
}
|
}
|
||||||
|
|
||||||
TupleTableSlot *resultSlot = ReturnTupleFromTuplestore(scanState);
|
return ReturnTupleFromTuplestore(scanState);
|
||||||
|
|
||||||
return resultSlot;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -596,6 +645,7 @@ AdaptiveExecutorCreateScan(CustomScan *scan)
|
||||||
scanState->distributedPlan = GetDistributedPlan(scan);
|
scanState->distributedPlan = GetDistributedPlan(scan);
|
||||||
|
|
||||||
scanState->customScanState.methods = &AdaptiveExecutorCustomExecMethods;
|
scanState->customScanState.methods = &AdaptiveExecutorCustomExecMethods;
|
||||||
|
scanState->PreExecScan = &CitusPreExecScan;
|
||||||
|
|
||||||
return (Node *) scanState;
|
return (Node *) scanState;
|
||||||
}
|
}
|
||||||
|
@ -726,8 +776,7 @@ CitusReScan(CustomScanState *node)
|
||||||
TupleDesc
|
TupleDesc
|
||||||
ScanStateGetTupleDescriptor(CitusScanState *scanState)
|
ScanStateGetTupleDescriptor(CitusScanState *scanState)
|
||||||
{
|
{
|
||||||
return scanState->customScanState.ss.ps.ps_ResultTupleSlot->
|
return scanState->customScanState.ss.ss_ScanTupleSlot->tts_tupleDescriptor;
|
||||||
tts_tupleDescriptor;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -21,6 +21,7 @@
|
||||||
#include "distributed/commands/utility_hook.h"
|
#include "distributed/commands/utility_hook.h"
|
||||||
#include "distributed/insert_select_executor.h"
|
#include "distributed/insert_select_executor.h"
|
||||||
#include "distributed/insert_select_planner.h"
|
#include "distributed/insert_select_planner.h"
|
||||||
|
#include "distributed/listutils.h"
|
||||||
#include "distributed/master_protocol.h"
|
#include "distributed/master_protocol.h"
|
||||||
#include "distributed/multi_executor.h"
|
#include "distributed/multi_executor.h"
|
||||||
#include "distributed/multi_master_planner.h"
|
#include "distributed/multi_master_planner.h"
|
||||||
|
@ -34,6 +35,7 @@
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
#include "executor/execdebug.h"
|
#include "executor/execdebug.h"
|
||||||
#include "commands/copy.h"
|
#include "commands/copy.h"
|
||||||
|
#include "nodes/execnodes.h"
|
||||||
#include "nodes/makefuncs.h"
|
#include "nodes/makefuncs.h"
|
||||||
#include "nodes/nodeFuncs.h"
|
#include "nodes/nodeFuncs.h"
|
||||||
#include "parser/parsetree.h"
|
#include "parser/parsetree.h"
|
||||||
|
@ -69,6 +71,9 @@ int ExecutorLevel = 0;
|
||||||
static Relation StubRelation(TupleDesc tupleDescriptor);
|
static Relation StubRelation(TupleDesc tupleDescriptor);
|
||||||
static bool AlterTableConstraintCheck(QueryDesc *queryDesc);
|
static bool AlterTableConstraintCheck(QueryDesc *queryDesc);
|
||||||
static bool IsLocalReferenceTableJoinPlan(PlannedStmt *plan);
|
static bool IsLocalReferenceTableJoinPlan(PlannedStmt *plan);
|
||||||
|
static List * FindCitusCustomScanStates(PlanState *planState);
|
||||||
|
static bool CitusCustomScanStateWalker(PlanState *planState,
|
||||||
|
List **citusCustomScanStates);
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* CitusExecutorStart is the ExecutorStart_hook that gets called when
|
* CitusExecutorStart is the ExecutorStart_hook that gets called when
|
||||||
|
@ -123,10 +128,25 @@ CitusExecutorRun(QueryDesc *queryDesc,
|
||||||
{
|
{
|
||||||
DestReceiver *dest = queryDesc->dest;
|
DestReceiver *dest = queryDesc->dest;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We do some potentially time consuming operations our self now before we hand of
|
||||||
|
* control to postgres' executor. To make sure that time spent is accurately measured
|
||||||
|
* we remove the totaltime instrumentation from the queryDesc. Instead we will start
|
||||||
|
* and stop the instrumentation of the total time and put it back on the queryDesc
|
||||||
|
* before returning (or rethrowing) from this function.
|
||||||
|
*/
|
||||||
|
Instrumentation *volatile totalTime = queryDesc->totaltime;
|
||||||
|
queryDesc->totaltime = NULL;
|
||||||
|
|
||||||
PG_TRY();
|
PG_TRY();
|
||||||
{
|
{
|
||||||
ExecutorLevel++;
|
ExecutorLevel++;
|
||||||
|
|
||||||
|
if (totalTime)
|
||||||
|
{
|
||||||
|
InstrStartNode(totalTime);
|
||||||
|
}
|
||||||
|
|
||||||
if (CitusHasBeenLoaded())
|
if (CitusHasBeenLoaded())
|
||||||
{
|
{
|
||||||
if (IsLocalReferenceTableJoinPlan(queryDesc->plannedstmt) &&
|
if (IsLocalReferenceTableJoinPlan(queryDesc->plannedstmt) &&
|
||||||
|
@ -174,13 +194,47 @@ CitusExecutorRun(QueryDesc *queryDesc,
|
||||||
}
|
}
|
||||||
else
|
else
|
||||||
{
|
{
|
||||||
|
/* switch into per-query memory context before calling PreExecScan */
|
||||||
|
MemoryContext oldcontext = MemoryContextSwitchTo(
|
||||||
|
queryDesc->estate->es_query_cxt);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Call PreExecScan for all citus custom scan nodes prior to starting the
|
||||||
|
* postgres exec scan to give some citus scan nodes some time to initialize
|
||||||
|
* state that would be too late if it were to initialize when the first tuple
|
||||||
|
* would need to return.
|
||||||
|
*/
|
||||||
|
List *citusCustomScanStates = FindCitusCustomScanStates(queryDesc->planstate);
|
||||||
|
CitusScanState *citusScanState = NULL;
|
||||||
|
foreach_ptr(citusScanState, citusCustomScanStates)
|
||||||
|
{
|
||||||
|
if (citusScanState->PreExecScan)
|
||||||
|
{
|
||||||
|
citusScanState->PreExecScan(citusScanState);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/* postgres will switch here again and will restore back on its own */
|
||||||
|
MemoryContextSwitchTo(oldcontext);
|
||||||
|
|
||||||
standard_ExecutorRun(queryDesc, direction, count, execute_once);
|
standard_ExecutorRun(queryDesc, direction, count, execute_once);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (totalTime)
|
||||||
|
{
|
||||||
|
InstrStopNode(totalTime, queryDesc->estate->es_processed);
|
||||||
|
queryDesc->totaltime = totalTime;
|
||||||
|
}
|
||||||
|
|
||||||
ExecutorLevel--;
|
ExecutorLevel--;
|
||||||
}
|
}
|
||||||
PG_CATCH();
|
PG_CATCH();
|
||||||
{
|
{
|
||||||
|
if (totalTime)
|
||||||
|
{
|
||||||
|
queryDesc->totaltime = totalTime;
|
||||||
|
}
|
||||||
|
|
||||||
ExecutorLevel--;
|
ExecutorLevel--;
|
||||||
|
|
||||||
PG_RE_THROW();
|
PG_RE_THROW();
|
||||||
|
@ -189,6 +243,38 @@ CitusExecutorRun(QueryDesc *queryDesc,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* FindCitusCustomScanStates returns a list of all citus custom scan states in it.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
FindCitusCustomScanStates(PlanState *planState)
|
||||||
|
{
|
||||||
|
List *citusCustomScanStates = NIL;
|
||||||
|
CitusCustomScanStateWalker(planState, &citusCustomScanStates);
|
||||||
|
return citusCustomScanStates;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CitusCustomScanStateWalker walks a planState tree structure and adds all
|
||||||
|
* CitusCustomState nodes to the list passed by reference as the second argument.
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
CitusCustomScanStateWalker(PlanState *planState, List **citusCustomScanStates)
|
||||||
|
{
|
||||||
|
if (IsCitusCustomState(planState))
|
||||||
|
{
|
||||||
|
CitusScanState *css = (CitusScanState *) planState;
|
||||||
|
*citusCustomScanStates = lappend(*citusCustomScanStates, css);
|
||||||
|
|
||||||
|
/* breaks the walking of this tree */
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
return planstate_tree_walker(planState, CitusCustomScanStateWalker,
|
||||||
|
citusCustomScanStates);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ReturnTupleFromTuplestore reads the next tuple from the tuple store of the
|
* ReturnTupleFromTuplestore reads the next tuple from the tuple store of the
|
||||||
* given Citus scan node and returns it. It returns null if all tuples are read
|
* given Citus scan node and returns it. It returns null if all tuples are read
|
||||||
|
@ -214,10 +300,77 @@ ReturnTupleFromTuplestore(CitusScanState *scanState)
|
||||||
forwardScanDirection = false;
|
forwardScanDirection = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
TupleTableSlot *resultSlot = scanState->customScanState.ss.ps.ps_ResultTupleSlot;
|
ExprState *qual = scanState->customScanState.ss.ps.qual;
|
||||||
tuplestore_gettupleslot(tupleStore, forwardScanDirection, false, resultSlot);
|
ProjectionInfo *projInfo = scanState->customScanState.ss.ps.ps_ProjInfo;
|
||||||
|
ExprContext *econtext = scanState->customScanState.ss.ps.ps_ExprContext;
|
||||||
|
|
||||||
return resultSlot;
|
if (!qual && !projInfo)
|
||||||
|
{
|
||||||
|
/* no quals, nor projections return directly from the tuple store. */
|
||||||
|
TupleTableSlot *slot = scanState->customScanState.ss.ss_ScanTupleSlot;
|
||||||
|
tuplestore_gettupleslot(tupleStore, forwardScanDirection, false, slot);
|
||||||
|
return slot;
|
||||||
|
}
|
||||||
|
|
||||||
|
for (;;)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* If there is a very selective qual on the Citus Scan node we might block
|
||||||
|
* interupts for a longer time if we would not check for interrupts in this loop
|
||||||
|
*/
|
||||||
|
CHECK_FOR_INTERRUPTS();
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Reset per-tuple memory context to free any expression evaluation
|
||||||
|
* storage allocated in the previous tuple cycle.
|
||||||
|
*/
|
||||||
|
ResetExprContext(econtext);
|
||||||
|
|
||||||
|
TupleTableSlot *slot = scanState->customScanState.ss.ss_ScanTupleSlot;
|
||||||
|
tuplestore_gettupleslot(tupleStore, forwardScanDirection, false, slot);
|
||||||
|
|
||||||
|
if (TupIsNull(slot))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* When the tuple is null we have reached the end of the tuplestore. We will
|
||||||
|
* return a null tuple, however, depending on the existence of a projection we
|
||||||
|
* need to either return the scan tuple or the projected tuple.
|
||||||
|
*/
|
||||||
|
if (projInfo)
|
||||||
|
{
|
||||||
|
return ExecClearTuple(projInfo->pi_state.resultslot);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
return slot;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/* place the current tuple into the expr context */
|
||||||
|
econtext->ecxt_scantuple = slot;
|
||||||
|
|
||||||
|
if (!ExecQual(qual, econtext))
|
||||||
|
{
|
||||||
|
/* skip nodes that do not satisfy the qual (filter) */
|
||||||
|
InstrCountFiltered1(scanState, 1);
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* found a satisfactory scan tuple */
|
||||||
|
if (projInfo)
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* Form a projection tuple, store it in the result tuple slot and return it.
|
||||||
|
* ExecProj works on the ecxt_scantuple on the context stored earlier.
|
||||||
|
*/
|
||||||
|
return ExecProject(projInfo);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
/* Here, we aren't projecting, so just return scan tuple */
|
||||||
|
return slot;
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -24,6 +24,7 @@
|
||||||
#include "distributed/insert_select_planner.h"
|
#include "distributed/insert_select_planner.h"
|
||||||
#include "distributed/intermediate_result_pruning.h"
|
#include "distributed/intermediate_result_pruning.h"
|
||||||
#include "distributed/intermediate_results.h"
|
#include "distributed/intermediate_results.h"
|
||||||
|
#include "distributed/listutils.h"
|
||||||
#include "distributed/master_protocol.h"
|
#include "distributed/master_protocol.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/multi_executor.h"
|
#include "distributed/multi_executor.h"
|
||||||
|
@ -53,6 +54,7 @@
|
||||||
#endif
|
#endif
|
||||||
#include "optimizer/pathnode.h"
|
#include "optimizer/pathnode.h"
|
||||||
#include "optimizer/planner.h"
|
#include "optimizer/planner.h"
|
||||||
|
#include "optimizer/planmain.h"
|
||||||
#include "utils/builtins.h"
|
#include "utils/builtins.h"
|
||||||
#include "utils/datum.h"
|
#include "utils/datum.h"
|
||||||
#include "utils/lsyscache.h"
|
#include "utils/lsyscache.h"
|
||||||
|
@ -98,6 +100,8 @@ static PlannedStmt * FinalizeNonRouterPlan(PlannedStmt *localPlan,
|
||||||
DistributedPlan *distributedPlan,
|
DistributedPlan *distributedPlan,
|
||||||
CustomScan *customScan);
|
CustomScan *customScan);
|
||||||
static PlannedStmt * FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan);
|
static PlannedStmt * FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan);
|
||||||
|
static List * makeTargetListFromCustomScanList(List *custom_scan_tlist);
|
||||||
|
static List * makeCustomScanTargetlistFromExistingTargetList(List *existingTargetlist);
|
||||||
static int32 BlessRecordExpressionList(List *exprs);
|
static int32 BlessRecordExpressionList(List *exprs);
|
||||||
static void CheckNodeIsDumpable(Node *node);
|
static void CheckNodeIsDumpable(Node *node);
|
||||||
static Node * CheckNodeCopyAndSerialization(Node *node);
|
static Node * CheckNodeCopyAndSerialization(Node *node);
|
||||||
|
@ -1425,18 +1429,58 @@ FinalizeNonRouterPlan(PlannedStmt *localPlan, DistributedPlan *distributedPlan,
|
||||||
static PlannedStmt *
|
static PlannedStmt *
|
||||||
FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan)
|
FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan)
|
||||||
{
|
{
|
||||||
ListCell *targetEntryCell = NULL;
|
|
||||||
List *targetList = NIL;
|
|
||||||
List *columnNameList = NIL;
|
List *columnNameList = NIL;
|
||||||
|
|
||||||
|
customScan->custom_scan_tlist =
|
||||||
|
makeCustomScanTargetlistFromExistingTargetList(localPlan->planTree->targetlist);
|
||||||
|
customScan->scan.plan.targetlist =
|
||||||
|
makeTargetListFromCustomScanList(customScan->custom_scan_tlist);
|
||||||
|
|
||||||
|
/* extract the column names from the final targetlist*/
|
||||||
|
TargetEntry *targetEntry = NULL;
|
||||||
|
foreach_ptr(targetEntry, customScan->scan.plan.targetlist)
|
||||||
|
{
|
||||||
|
Value *columnName = makeString(targetEntry->resname);
|
||||||
|
columnNameList = lappend(columnNameList, columnName);
|
||||||
|
}
|
||||||
|
|
||||||
|
PlannedStmt *routerPlan = makeNode(PlannedStmt);
|
||||||
|
routerPlan->planTree = (Plan *) customScan;
|
||||||
|
|
||||||
|
RangeTblEntry *remoteScanRangeTableEntry = RemoteScanRangeTableEntry(columnNameList);
|
||||||
|
routerPlan->rtable = list_make1(remoteScanRangeTableEntry);
|
||||||
|
|
||||||
|
/* add original range table list for access permission checks */
|
||||||
|
routerPlan->rtable = list_concat(routerPlan->rtable, localPlan->rtable);
|
||||||
|
|
||||||
|
routerPlan->canSetTag = true;
|
||||||
|
routerPlan->relationOids = NIL;
|
||||||
|
|
||||||
|
routerPlan->queryId = localPlan->queryId;
|
||||||
|
routerPlan->utilityStmt = localPlan->utilityStmt;
|
||||||
|
routerPlan->commandType = localPlan->commandType;
|
||||||
|
routerPlan->hasReturning = localPlan->hasReturning;
|
||||||
|
|
||||||
|
return routerPlan;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* makeCustomScanTargetlistFromExistingTargetList rebuilds the targetlist from the remote
|
||||||
|
* query into a list that can be used as the custom_scan_tlist for our Citus Custom Scan.
|
||||||
|
*/
|
||||||
|
static List *
|
||||||
|
makeCustomScanTargetlistFromExistingTargetList(List *existingTargetlist)
|
||||||
|
{
|
||||||
|
List *custom_scan_tlist = NIL;
|
||||||
|
|
||||||
/* we will have custom scan range table entry as the first one in the list */
|
/* we will have custom scan range table entry as the first one in the list */
|
||||||
int customScanRangeTableIndex = 1;
|
const int customScanRangeTableIndex = 1;
|
||||||
|
|
||||||
/* build a targetlist to read from the custom scan output */
|
/* build a targetlist to read from the custom scan output */
|
||||||
foreach(targetEntryCell, localPlan->planTree->targetlist)
|
TargetEntry *targetEntry = NULL;
|
||||||
|
foreach_ptr(targetEntry, existingTargetlist)
|
||||||
{
|
{
|
||||||
TargetEntry *targetEntry = lfirst(targetEntryCell);
|
|
||||||
|
|
||||||
Assert(IsA(targetEntry, TargetEntry));
|
Assert(IsA(targetEntry, TargetEntry));
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -1465,32 +1509,40 @@ FinalizeRouterPlan(PlannedStmt *localPlan, CustomScan *customScan)
|
||||||
|
|
||||||
TargetEntry *newTargetEntry = flatCopyTargetEntry(targetEntry);
|
TargetEntry *newTargetEntry = flatCopyTargetEntry(targetEntry);
|
||||||
newTargetEntry->expr = (Expr *) newVar;
|
newTargetEntry->expr = (Expr *) newVar;
|
||||||
targetList = lappend(targetList, newTargetEntry);
|
custom_scan_tlist = lappend(custom_scan_tlist, newTargetEntry);
|
||||||
|
|
||||||
Value *columnName = makeString(targetEntry->resname);
|
|
||||||
columnNameList = lappend(columnNameList, columnName);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
customScan->scan.plan.targetlist = targetList;
|
return custom_scan_tlist;
|
||||||
|
}
|
||||||
|
|
||||||
PlannedStmt *routerPlan = makeNode(PlannedStmt);
|
|
||||||
routerPlan->planTree = (Plan *) customScan;
|
|
||||||
|
|
||||||
RangeTblEntry *remoteScanRangeTableEntry = RemoteScanRangeTableEntry(columnNameList);
|
/*
|
||||||
routerPlan->rtable = list_make1(remoteScanRangeTableEntry);
|
* makeTargetListFromCustomScanList based on a custom_scan_tlist create the target list to
|
||||||
|
* use on the Citus Custom Scan Node. The targetlist differs from the custom_scan_tlist in
|
||||||
/* add original range table list for access permission checks */
|
* a way that the expressions in the targetlist all are references to the index (resno) in
|
||||||
routerPlan->rtable = list_concat(routerPlan->rtable, localPlan->rtable);
|
* the custom_scan_tlist in their varattno while the varno is replaced with INDEX_VAR
|
||||||
|
* instead of the range table entry index.
|
||||||
routerPlan->canSetTag = true;
|
*/
|
||||||
routerPlan->relationOids = NIL;
|
static List *
|
||||||
|
makeTargetListFromCustomScanList(List *custom_scan_tlist)
|
||||||
routerPlan->queryId = localPlan->queryId;
|
{
|
||||||
routerPlan->utilityStmt = localPlan->utilityStmt;
|
List *targetList = NIL;
|
||||||
routerPlan->commandType = localPlan->commandType;
|
TargetEntry *targetEntry = NULL;
|
||||||
routerPlan->hasReturning = localPlan->hasReturning;
|
int resno = 1;
|
||||||
|
foreach_ptr(targetEntry, custom_scan_tlist)
|
||||||
return routerPlan;
|
{
|
||||||
|
/*
|
||||||
|
* INDEX_VAR is used to reference back to the TargetEntry in custom_scan_tlist by
|
||||||
|
* its resno (index)
|
||||||
|
*/
|
||||||
|
Var *newVar = makeVarFromTargetEntry(INDEX_VAR, targetEntry);
|
||||||
|
TargetEntry *newTargetEntry = makeTargetEntry((Expr *) newVar, resno,
|
||||||
|
targetEntry->resname,
|
||||||
|
targetEntry->resjunk);
|
||||||
|
targetList = lappend(targetList, newTargetEntry);
|
||||||
|
resno++;
|
||||||
|
}
|
||||||
|
return targetList;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -1779,6 +1831,25 @@ multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
|
||||||
{
|
{
|
||||||
DistTableCacheEntry *cacheEntry = NULL;
|
DistTableCacheEntry *cacheEntry = NULL;
|
||||||
|
|
||||||
|
if (ReplaceCitusExtraDataContainer && IsCitusExtraDataContainerRelation(rte))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* We got here by planning the query part that needs to be executed on the query
|
||||||
|
* coordinator node.
|
||||||
|
* We have verified the occurrence of the citus_extra_datacontainer function
|
||||||
|
* encoding the remote scan we plan to execute here. We will replace all paths
|
||||||
|
* with a path describing our custom scan.
|
||||||
|
*/
|
||||||
|
Path *path = CreateCitusCustomScanPath(root, relOptInfo, restrictionIndex, rte,
|
||||||
|
ReplaceCitusExtraDataContainerWithCustomScan);
|
||||||
|
|
||||||
|
/* replace all paths with our custom scan and recalculate cheapest */
|
||||||
|
relOptInfo->pathlist = list_make1(path);
|
||||||
|
set_cheapest(relOptInfo);
|
||||||
|
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
|
||||||
AdjustReadIntermediateResultCost(rte, relOptInfo);
|
AdjustReadIntermediateResultCost(rte, relOptInfo);
|
||||||
AdjustReadIntermediateResultArrayCost(rte, relOptInfo);
|
AdjustReadIntermediateResultArrayCost(rte, relOptInfo);
|
||||||
|
|
||||||
|
|
|
@ -1485,6 +1485,15 @@ MasterExtendedOpNode(MultiExtendedOp *originalOpNode,
|
||||||
if (originalHavingQual != NULL)
|
if (originalHavingQual != NULL)
|
||||||
{
|
{
|
||||||
newHavingQual = MasterAggregateMutator(originalHavingQual, &walkerContext);
|
newHavingQual = MasterAggregateMutator(originalHavingQual, &walkerContext);
|
||||||
|
if (IsA(newHavingQual, List))
|
||||||
|
{
|
||||||
|
/*
|
||||||
|
* unflatten having qual to allow standard planner to work when transforming
|
||||||
|
* the master query to a plan
|
||||||
|
*/
|
||||||
|
newHavingQual = (Node *) make_ands_explicit(
|
||||||
|
castNode(List, newHavingQual));
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -79,6 +79,7 @@ static bool HasTablesample(Query *queryTree);
|
||||||
static bool HasComplexRangeTableType(Query *queryTree);
|
static bool HasComplexRangeTableType(Query *queryTree);
|
||||||
static bool IsReadIntermediateResultFunction(Node *node);
|
static bool IsReadIntermediateResultFunction(Node *node);
|
||||||
static bool IsReadIntermediateResultArrayFunction(Node *node);
|
static bool IsReadIntermediateResultArrayFunction(Node *node);
|
||||||
|
static bool IsCitusExtraDataContainerFunc(Node *node);
|
||||||
static bool IsFunctionWithOid(Node *node, Oid funcOid);
|
static bool IsFunctionWithOid(Node *node, Oid funcOid);
|
||||||
static bool ExtractFromExpressionWalker(Node *node,
|
static bool ExtractFromExpressionWalker(Node *node,
|
||||||
QualifierWalkerContext *walkerContext);
|
QualifierWalkerContext *walkerContext);
|
||||||
|
@ -811,6 +812,39 @@ IsReadIntermediateResultArrayFunction(Node *node)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* IsCitusExtraDataContainerRelation determines whether a range table entry contains a
|
||||||
|
* call to the citus_extradata_container function.
|
||||||
|
*/
|
||||||
|
bool
|
||||||
|
IsCitusExtraDataContainerRelation(RangeTblEntry *rte)
|
||||||
|
{
|
||||||
|
if (rte->rtekind != RTE_FUNCTION || list_length(rte->functions) != 1)
|
||||||
|
{
|
||||||
|
/* avoid more expensive checks below for non-functions */
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (!CitusHasBeenLoaded() || !CheckCitusVersion(DEBUG5))
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
return FindNodeCheck((Node *) rte->functions, IsCitusExtraDataContainerFunc);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* IsCitusExtraDataContainerFunc determines whether a given node is a function call
|
||||||
|
* to the citus_extradata_container function.
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
IsCitusExtraDataContainerFunc(Node *node)
|
||||||
|
{
|
||||||
|
return IsFunctionWithOid(node, CitusExtraDataContainerFuncId());
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* IsFunctionWithOid determines whether a given node is a function call
|
* IsFunctionWithOid determines whether a given node is a function call
|
||||||
* to the read_intermediate_result function.
|
* to the read_intermediate_result function.
|
||||||
|
|
|
@ -14,50 +14,38 @@
|
||||||
#include "postgres.h"
|
#include "postgres.h"
|
||||||
|
|
||||||
#include "catalog/pg_type.h"
|
#include "catalog/pg_type.h"
|
||||||
#include "commands/extension.h"
|
|
||||||
#include "distributed/citus_ruleutils.h"
|
#include "distributed/citus_ruleutils.h"
|
||||||
#include "distributed/function_utils.h"
|
|
||||||
#include "distributed/listutils.h"
|
#include "distributed/listutils.h"
|
||||||
#include "distributed/multi_logical_optimizer.h"
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/multi_master_planner.h"
|
#include "distributed/multi_master_planner.h"
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
#include "distributed/distributed_planner.h"
|
|
||||||
#include "distributed/multi_server_executor.h"
|
|
||||||
#include "distributed/version_compat.h"
|
|
||||||
#include "distributed/worker_protocol.h"
|
|
||||||
#include "nodes/makefuncs.h"
|
#include "nodes/makefuncs.h"
|
||||||
#include "nodes/nodeFuncs.h"
|
#include "nodes/nodeFuncs.h"
|
||||||
#include "nodes/print.h"
|
#include "optimizer/planner.h"
|
||||||
#include "optimizer/clauses.h"
|
#include "rewrite/rewriteManip.h"
|
||||||
#include "optimizer/cost.h"
|
|
||||||
#include "optimizer/planmain.h"
|
|
||||||
#include "optimizer/tlist.h"
|
|
||||||
#include "optimizer/subselect.h"
|
|
||||||
#if PG_VERSION_NUM >= 120000
|
|
||||||
#include "optimizer/optimizer.h"
|
|
||||||
#else
|
|
||||||
#include "optimizer/var.h"
|
|
||||||
#endif
|
|
||||||
#include "utils/builtins.h"
|
|
||||||
#include "utils/guc.h"
|
|
||||||
#include "utils/memutils.h"
|
|
||||||
#include "utils/rel.h"
|
|
||||||
#include "utils/syscache.h"
|
|
||||||
#include "utils/lsyscache.h"
|
|
||||||
|
|
||||||
|
|
||||||
static List * MasterTargetList(List *workerTargetList);
|
static List * MasterTargetList(List *workerTargetList);
|
||||||
static PlannedStmt * BuildSelectStatement(Query *masterQuery, List *masterTargetList,
|
static PlannedStmt * BuildSelectStatementViaStdPlanner(Query *masterQuery,
|
||||||
|
List *masterTargetList,
|
||||||
CustomScan *remoteScan);
|
CustomScan *remoteScan);
|
||||||
static Agg * BuildAggregatePlan(PlannerInfo *root, Query *masterQuery, Plan *subPlan);
|
static bool FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result);
|
||||||
static bool HasDistinctOrOrderByAggregate(Query *masterQuery);
|
|
||||||
static bool UseGroupAggregateWithHLL(Query *masterQuery);
|
|
||||||
static bool QueryContainsAggregateWithHLL(Query *query);
|
|
||||||
static Plan * BuildDistinctPlan(Query *masterQuery, Plan *subPlan);
|
|
||||||
static Agg * makeAggNode(List *groupClauseList, List *havingQual,
|
|
||||||
AggStrategy aggrStrategy, List *queryTargetList, Plan *subPlan);
|
|
||||||
static void FinalizeStatement(PlannerInfo *root, PlannedStmt *stmt, Plan *topLevelPlan);
|
|
||||||
|
|
||||||
|
static Plan * CitusCustomScanPathPlan(PlannerInfo *root, RelOptInfo *rel,
|
||||||
|
struct CustomPath *best_path, List *tlist,
|
||||||
|
List *clauses, List *custom_plans);
|
||||||
|
|
||||||
|
bool ReplaceCitusExtraDataContainer = false;
|
||||||
|
CustomScan *ReplaceCitusExtraDataContainerWithCustomScan = NULL;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CitusCustomScanPathMethods defines the methods for a custom path we insert into the
|
||||||
|
* planner during the planning of the query part that will be executed on the node
|
||||||
|
* coordinating the query.
|
||||||
|
*/
|
||||||
|
static CustomPathMethods CitusCustomScanPathMethods = {
|
||||||
|
.CustomName = "CitusCustomScanPath",
|
||||||
|
.PlanCustomPath = CitusCustomScanPathPlan,
|
||||||
|
};
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* MasterNodeSelectPlan takes in a distributed plan and a custom scan node which
|
* MasterNodeSelectPlan takes in a distributed plan and a custom scan node which
|
||||||
|
@ -75,11 +63,7 @@ MasterNodeSelectPlan(DistributedPlan *distributedPlan, CustomScan *remoteScan)
|
||||||
Job *workerJob = distributedPlan->workerJob;
|
Job *workerJob = distributedPlan->workerJob;
|
||||||
List *workerTargetList = workerJob->jobQuery->targetList;
|
List *workerTargetList = workerJob->jobQuery->targetList;
|
||||||
List *masterTargetList = MasterTargetList(workerTargetList);
|
List *masterTargetList = MasterTargetList(workerTargetList);
|
||||||
|
return BuildSelectStatementViaStdPlanner(masterQuery, masterTargetList, remoteScan);
|
||||||
PlannedStmt *masterSelectPlan = BuildSelectStatement(masterQuery, masterTargetList,
|
|
||||||
remoteScan);
|
|
||||||
|
|
||||||
return masterSelectPlan;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
@ -132,516 +116,199 @@ MasterTargetList(List *workerTargetList)
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* BuildSelectStatement builds the final select statement to run on the master
|
* CreateCitusCustomScanPath creates a custom path node that will return the CustomScan if
|
||||||
* node, before returning results to the user. The function first gets the custom
|
* the path ends up in the best_path during postgres planning. We use this function during
|
||||||
* scan node for all results fetched to the master, and layers aggregation, sort
|
* the set relation hook of postgres during the planning of the query part that will be
|
||||||
* and limit plans on top of the scan statement if necessary.
|
* executed on the query coordinating node.
|
||||||
|
*/
|
||||||
|
Path *
|
||||||
|
CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo,
|
||||||
|
Index restrictionIndex, RangeTblEntry *rte,
|
||||||
|
CustomScan *remoteScan)
|
||||||
|
{
|
||||||
|
CitusCustomScanPath *path = (CitusCustomScanPath *) newNode(
|
||||||
|
sizeof(CitusCustomScanPath), T_CustomPath);
|
||||||
|
path->custom_path.methods = &CitusCustomScanPathMethods;
|
||||||
|
path->custom_path.path.pathtype = T_CustomScan;
|
||||||
|
path->custom_path.path.pathtarget = relOptInfo->reltarget;
|
||||||
|
path->custom_path.path.parent = relOptInfo;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* The 100k rows we put on the cost of the path is kind of arbitrary and could be
|
||||||
|
* improved in accuracy to produce better plans.
|
||||||
|
*
|
||||||
|
* 100k on the row estimate causes the postgres planner to behave very much like the
|
||||||
|
* old citus planner in the plans it produces. Namely the old planner had hardcoded
|
||||||
|
* the use of Hash Aggregates for most of the operations, unless a postgres guc was
|
||||||
|
* set that would disallow hash aggregates to be used.
|
||||||
|
*
|
||||||
|
* Ideally we would be able to provide estimates close to postgres' estimates on the
|
||||||
|
* workers to let the standard planner choose an optimal solution for the masterQuery.
|
||||||
|
*/
|
||||||
|
path->custom_path.path.rows = 100000;
|
||||||
|
path->remoteScan = remoteScan;
|
||||||
|
|
||||||
|
return (Path *) path;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CitusCustomScanPathPlan is called for the CitusCustomScanPath node in the best_path
|
||||||
|
* after the postgres planner has evaluated all possible paths.
|
||||||
|
*
|
||||||
|
* This function returns a Plan node, more specifically the CustomScan Plan node that has
|
||||||
|
* the ability to execute the distributed part of the query.
|
||||||
|
*
|
||||||
|
* When this function is called there is an extra list of clauses passed in that might not
|
||||||
|
* already have been applied to the plan. We add these clauses to the quals this node will
|
||||||
|
* execute. The quals are evaluated before returning the tuples scanned from the workers
|
||||||
|
* to the plan above ours to make sure they do not end up in the final result.
|
||||||
|
*/
|
||||||
|
static Plan *
|
||||||
|
CitusCustomScanPathPlan(PlannerInfo *root,
|
||||||
|
RelOptInfo *rel,
|
||||||
|
struct CustomPath *best_path,
|
||||||
|
List *tlist,
|
||||||
|
List *clauses,
|
||||||
|
List *custom_plans)
|
||||||
|
{
|
||||||
|
CitusCustomScanPath *citusPath = (CitusCustomScanPath *) best_path;
|
||||||
|
|
||||||
|
/* clauses might have been added by the planner, need to add them to our scan */
|
||||||
|
RestrictInfo *restrictInfo = NULL;
|
||||||
|
List **quals = &citusPath->remoteScan->scan.plan.qual;
|
||||||
|
foreach_ptr(restrictInfo, clauses)
|
||||||
|
{
|
||||||
|
*quals = lappend(*quals, restrictInfo->clause);
|
||||||
|
}
|
||||||
|
return (Plan *) citusPath->remoteScan;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* BuildSelectStatementViaStdPlanner creates a PlannedStmt where it combines the
|
||||||
|
* masterQuery and the remoteScan. It utilizes the standard_planner from postgres to
|
||||||
|
* create a plan based on the masterQuery.
|
||||||
*/
|
*/
|
||||||
static PlannedStmt *
|
static PlannedStmt *
|
||||||
BuildSelectStatement(Query *masterQuery, List *masterTargetList, CustomScan *remoteScan)
|
BuildSelectStatementViaStdPlanner(Query *masterQuery, List *masterTargetList,
|
||||||
|
CustomScan *remoteScan)
|
||||||
{
|
{
|
||||||
/* top level select query should have only one range table entry */
|
/*
|
||||||
Assert(list_length(masterQuery->rtable) == 1);
|
* the standard planner will scribble on the target list. Since it is essential to not
|
||||||
Agg *aggregationPlan = NULL;
|
* change the custom_scan_tlist we copy the target list before adding them to any.
|
||||||
Plan *topLevelPlan = NULL;
|
* The masterTargetList is used in the end to extract the column names to be added to
|
||||||
List *sortClauseList = copyObject(masterQuery->sortClause);
|
* the alias we will create for the CustomScan, (expressed as the
|
||||||
|
* citus_extradata_container function call in the masterQuery).
|
||||||
|
*/
|
||||||
|
remoteScan->custom_scan_tlist = copyObject(masterTargetList);
|
||||||
|
remoteScan->scan.plan.targetlist = copyObject(masterTargetList);
|
||||||
|
|
||||||
|
/* probably want to do this where we add sublinks to the master plan */
|
||||||
|
masterQuery->hasSubLinks = checkExprHasSubLink((Node *) masterQuery);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* We will overwrite the alias of the rangetable which describes the custom scan.
|
||||||
|
* Idealy we would have set the correct column names and alias on the range table in
|
||||||
|
* the master query already when we inserted the extra data container. This could be
|
||||||
|
* improved in the future.
|
||||||
|
*/
|
||||||
|
|
||||||
|
/* find the rangetable entry for the extradata container and overwrite its alias */
|
||||||
|
RangeTblEntry *extradataContainerRTE = NULL;
|
||||||
|
FindCitusExtradataContainerRTE((Node *) masterQuery, &extradataContainerRTE);
|
||||||
|
if (extradataContainerRTE != NULL)
|
||||||
|
{
|
||||||
|
/* extract column names from the masterTargetList */
|
||||||
List *columnNameList = NIL;
|
List *columnNameList = NIL;
|
||||||
TargetEntry *targetEntry = NULL;
|
TargetEntry *targetEntry = NULL;
|
||||||
|
|
||||||
PlannerGlobal *glob = makeNode(PlannerGlobal);
|
|
||||||
PlannerInfo *root = makeNode(PlannerInfo);
|
|
||||||
root->parse = masterQuery;
|
|
||||||
root->glob = glob;
|
|
||||||
root->query_level = 1;
|
|
||||||
root->planner_cxt = CurrentMemoryContext;
|
|
||||||
root->wt_param_id = -1;
|
|
||||||
|
|
||||||
|
|
||||||
/* (1) make PlannedStmt and set basic information */
|
|
||||||
PlannedStmt *selectStatement = makeNode(PlannedStmt);
|
|
||||||
selectStatement->canSetTag = true;
|
|
||||||
selectStatement->relationOids = NIL;
|
|
||||||
selectStatement->commandType = CMD_SELECT;
|
|
||||||
|
|
||||||
|
|
||||||
remoteScan->custom_scan_tlist = masterTargetList;
|
|
||||||
|
|
||||||
/* (2) add an aggregation plan if needed */
|
|
||||||
if (masterQuery->hasAggs || masterQuery->groupClause)
|
|
||||||
{
|
|
||||||
remoteScan->scan.plan.targetlist = masterTargetList;
|
|
||||||
|
|
||||||
aggregationPlan = BuildAggregatePlan(root, masterQuery, &remoteScan->scan.plan);
|
|
||||||
topLevelPlan = (Plan *) aggregationPlan;
|
|
||||||
selectStatement->planTree = topLevelPlan;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
/* otherwise set the final projections on the scan plan directly */
|
|
||||||
|
|
||||||
/*
|
|
||||||
* The masterTargetList contains all columns that we fetch from
|
|
||||||
* the worker as non-resjunk.
|
|
||||||
*
|
|
||||||
* Here the output of the plan node determines the output of the query.
|
|
||||||
* We therefore use the targetList of masterQuery, which has non-output
|
|
||||||
* columns set as resjunk.
|
|
||||||
*/
|
|
||||||
remoteScan->scan.plan.targetlist = masterQuery->targetList;
|
|
||||||
topLevelPlan = &remoteScan->scan.plan;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* (3) create distinct plan if needed.
|
|
||||||
*
|
|
||||||
* distinct on() requires sort + unique plans. Unique itself is not enough
|
|
||||||
* as it only compares the current value with previous one when checking
|
|
||||||
* uniqueness, thus ordering is necessary. If already has order by
|
|
||||||
* clause we append distinct clauses to the end of it. Postgresql requires
|
|
||||||
* that if both distinct on() and order by exists, ordering shall start
|
|
||||||
* on distinct clauses. Therefore we can safely append distinct clauses to
|
|
||||||
* the end of order by clauses. Although the same column may appear more
|
|
||||||
* than once in order by clauses, created plan uses only one instance, for
|
|
||||||
* example order by a,b,a,a,b,c is translated to equivalent order by a,b,c.
|
|
||||||
*
|
|
||||||
* If the query has distinct clause but not distinct on, we first create
|
|
||||||
* distinct plan that is either HashAggreate or Sort + Unique plans depending
|
|
||||||
* on hashable property of columns in distinct clause. If there is order by
|
|
||||||
* clause, it is handled after distinct planning.
|
|
||||||
*/
|
|
||||||
if (masterQuery->hasDistinctOn)
|
|
||||||
{
|
|
||||||
ListCell *distinctCell = NULL;
|
|
||||||
foreach(distinctCell, masterQuery->distinctClause)
|
|
||||||
{
|
|
||||||
SortGroupClause *singleDistinctClause = lfirst(distinctCell);
|
|
||||||
Index sortGroupRef = singleDistinctClause->tleSortGroupRef;
|
|
||||||
|
|
||||||
if (get_sortgroupref_clause_noerr(sortGroupRef, sortClauseList) == NULL)
|
|
||||||
{
|
|
||||||
sortClauseList = lappend(sortClauseList, singleDistinctClause);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else if (masterQuery->distinctClause)
|
|
||||||
{
|
|
||||||
Plan *distinctPlan = BuildDistinctPlan(masterQuery, topLevelPlan);
|
|
||||||
topLevelPlan = distinctPlan;
|
|
||||||
}
|
|
||||||
|
|
||||||
/* (4) add a sorting plan if needed */
|
|
||||||
if (sortClauseList)
|
|
||||||
{
|
|
||||||
Sort *sortPlan = make_sort_from_sortclauses(sortClauseList, topLevelPlan);
|
|
||||||
|
|
||||||
/* just for reproducible costs between different PostgreSQL versions */
|
|
||||||
sortPlan->plan.startup_cost = 0;
|
|
||||||
sortPlan->plan.total_cost = 0;
|
|
||||||
sortPlan->plan.plan_rows = 0;
|
|
||||||
|
|
||||||
topLevelPlan = (Plan *) sortPlan;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* (5) add a unique plan for distinctOn.
|
|
||||||
* If the query has distinct on we add a sort clause in step 3. Therefore
|
|
||||||
* Step 4 always creates a sort plan.
|
|
||||||
* */
|
|
||||||
if (masterQuery->hasDistinctOn)
|
|
||||||
{
|
|
||||||
Assert(IsA(topLevelPlan, Sort));
|
|
||||||
topLevelPlan =
|
|
||||||
(Plan *) make_unique_from_sortclauses(topLevelPlan,
|
|
||||||
masterQuery->distinctClause);
|
|
||||||
}
|
|
||||||
|
|
||||||
/* (5) add a limit plan if needed */
|
|
||||||
if (masterQuery->limitCount || masterQuery->limitOffset)
|
|
||||||
{
|
|
||||||
Node *limitCount = masterQuery->limitCount;
|
|
||||||
Node *limitOffset = masterQuery->limitOffset;
|
|
||||||
Limit *limitPlan = make_limit(topLevelPlan, limitOffset, limitCount);
|
|
||||||
topLevelPlan = (Plan *) limitPlan;
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* (6) set top level plan in the plantree and copy over some things from
|
|
||||||
* PlannerInfo
|
|
||||||
*/
|
|
||||||
FinalizeStatement(root, selectStatement, topLevelPlan);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* (7) Replace rangetable with one with nice names to show in EXPLAIN plans
|
|
||||||
*/
|
|
||||||
foreach_ptr(targetEntry, masterTargetList)
|
foreach_ptr(targetEntry, masterTargetList)
|
||||||
{
|
{
|
||||||
columnNameList = lappend(columnNameList, makeString(targetEntry->resname));
|
columnNameList = lappend(columnNameList, makeString(targetEntry->resname));
|
||||||
}
|
}
|
||||||
|
extradataContainerRTE->eref = makeAlias("remote_scan", columnNameList);
|
||||||
|
}
|
||||||
|
|
||||||
RangeTblEntry *customScanRangeTableEntry = linitial(selectStatement->rtable);
|
/*
|
||||||
customScanRangeTableEntry->eref = makeAlias("remote_scan", columnNameList);
|
* Print the master query at debug level 4. Since serializing the query is relatively
|
||||||
|
* cpu intensive we only perform that if we are actually logging DEBUG4.
|
||||||
|
*/
|
||||||
|
const int logMasterQueryLevel = DEBUG4;
|
||||||
|
if (IsLoggableLevel(logMasterQueryLevel))
|
||||||
|
{
|
||||||
|
StringInfo queryString = makeStringInfo();
|
||||||
|
pg_get_query_def(masterQuery, queryString);
|
||||||
|
elog(logMasterQueryLevel, "master query: %s", queryString->data);
|
||||||
|
}
|
||||||
|
|
||||||
return selectStatement;
|
PlannedStmt *standardStmt = NULL;
|
||||||
|
PG_TRY();
|
||||||
|
{
|
||||||
|
/* This code should not be re-entrant, we check via asserts below */
|
||||||
|
Assert(ReplaceCitusExtraDataContainer == false);
|
||||||
|
Assert(ReplaceCitusExtraDataContainerWithCustomScan == NULL);
|
||||||
|
ReplaceCitusExtraDataContainer = true;
|
||||||
|
ReplaceCitusExtraDataContainerWithCustomScan = remoteScan;
|
||||||
|
|
||||||
|
standardStmt = standard_planner(masterQuery, 0, NULL);
|
||||||
|
|
||||||
|
ReplaceCitusExtraDataContainer = false;
|
||||||
|
ReplaceCitusExtraDataContainerWithCustomScan = NULL;
|
||||||
|
}
|
||||||
|
PG_CATCH();
|
||||||
|
{
|
||||||
|
ReplaceCitusExtraDataContainer = false;
|
||||||
|
ReplaceCitusExtraDataContainerWithCustomScan = NULL;
|
||||||
|
PG_RE_THROW();
|
||||||
|
}
|
||||||
|
PG_END_TRY();
|
||||||
|
|
||||||
|
Assert(standardStmt != NULL);
|
||||||
|
return standardStmt;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* FinalizeStatement sets some necessary fields on the final statement and its
|
* Finds the rangetable entry in the query that refers to the citus_extradata_container
|
||||||
* plan to make it work with the regular postgres executor. This code is copied
|
* and stores the pointer in result.
|
||||||
* almost verbatim from standard_planner in the PG source code.
|
|
||||||
*
|
|
||||||
* Modifications from original code:
|
|
||||||
* - Added SS_attach_initplans call
|
|
||||||
*/
|
|
||||||
static void
|
|
||||||
FinalizeStatement(PlannerInfo *root, PlannedStmt *result, Plan *top_plan)
|
|
||||||
{
|
|
||||||
ListCell *lp,
|
|
||||||
*lr;
|
|
||||||
PlannerGlobal *glob = root->glob;
|
|
||||||
|
|
||||||
/* Taken from create_plan */
|
|
||||||
SS_attach_initplans(root, top_plan);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* If any Params were generated, run through the plan tree and compute
|
|
||||||
* each plan node's extParam/allParam sets. Ideally we'd merge this into
|
|
||||||
* set_plan_references' tree traversal, but for now it has to be separate
|
|
||||||
* because we need to visit subplans before not after main plan.
|
|
||||||
*/
|
|
||||||
if (glob->paramExecTypes != NIL)
|
|
||||||
{
|
|
||||||
Assert(list_length(glob->subplans) == list_length(glob->subroots));
|
|
||||||
forboth(lp, glob->subplans, lr, glob->subroots)
|
|
||||||
{
|
|
||||||
Plan *subplan = (Plan *) lfirst(lp);
|
|
||||||
PlannerInfo *subroot = lfirst_node(PlannerInfo, lr);
|
|
||||||
|
|
||||||
SS_finalize_plan(subroot, subplan);
|
|
||||||
}
|
|
||||||
SS_finalize_plan(root, top_plan);
|
|
||||||
}
|
|
||||||
|
|
||||||
/* final cleanup of the plan */
|
|
||||||
Assert(glob->finalrtable == NIL);
|
|
||||||
Assert(glob->finalrowmarks == NIL);
|
|
||||||
Assert(glob->resultRelations == NIL);
|
|
||||||
Assert(glob->rootResultRelations == NIL);
|
|
||||||
|
|
||||||
top_plan = set_plan_references(root, top_plan);
|
|
||||||
|
|
||||||
/* ... and the subplans (both regular subplans and initplans) */
|
|
||||||
Assert(list_length(glob->subplans) == list_length(glob->subroots));
|
|
||||||
forboth(lp, glob->subplans, lr, glob->subroots)
|
|
||||||
{
|
|
||||||
Plan *subplan = (Plan *) lfirst(lp);
|
|
||||||
PlannerInfo *subroot = lfirst_node(PlannerInfo, lr);
|
|
||||||
|
|
||||||
lfirst(lp) = set_plan_references(subroot, subplan);
|
|
||||||
}
|
|
||||||
result->transientPlan = glob->transientPlan;
|
|
||||||
result->dependsOnRole = glob->dependsOnRole;
|
|
||||||
result->parallelModeNeeded = glob->parallelModeNeeded;
|
|
||||||
result->planTree = top_plan;
|
|
||||||
|
|
||||||
result->rtable = glob->finalrtable;
|
|
||||||
result->resultRelations = glob->resultRelations;
|
|
||||||
#if PG_VERSION_NUM < 120000
|
|
||||||
result->nonleafResultRelations = glob->nonleafResultRelations;
|
|
||||||
#endif
|
|
||||||
result->rootResultRelations = glob->rootResultRelations;
|
|
||||||
result->subplans = glob->subplans;
|
|
||||||
result->rewindPlanIDs = glob->rewindPlanIDs;
|
|
||||||
result->rowMarks = glob->finalrowmarks;
|
|
||||||
result->relationOids = glob->relationOids;
|
|
||||||
result->invalItems = glob->invalItems;
|
|
||||||
result->paramExecTypes = glob->paramExecTypes;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* BuildAggregatePlan creates and returns an aggregate plan. This aggregate plan
|
|
||||||
* builds aggregation and grouping operators (if any) that are to be executed on
|
|
||||||
* the master node.
|
|
||||||
*/
|
|
||||||
static Agg *
|
|
||||||
BuildAggregatePlan(PlannerInfo *root, Query *masterQuery, Plan *subPlan)
|
|
||||||
{
|
|
||||||
/* assert that we need to build an aggregate plan */
|
|
||||||
Assert(masterQuery->hasAggs || masterQuery->groupClause);
|
|
||||||
AggClauseCosts aggregateCosts;
|
|
||||||
AggStrategy aggregateStrategy = AGG_PLAIN;
|
|
||||||
List *groupColumnList = masterQuery->groupClause;
|
|
||||||
List *aggregateTargetList = masterQuery->targetList;
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Replaces SubLink nodes with SubPlan nodes in the having section of the
|
|
||||||
* query. (and creates the subplans in root->subplans)
|
|
||||||
*
|
|
||||||
* Would be nice if we could use masterQuery->hasSubLinks to only call
|
|
||||||
* these when that is true. However, for some reason hasSubLinks is false
|
|
||||||
* even when there are SubLinks.
|
|
||||||
*/
|
|
||||||
Node *havingQual = SS_process_sublinks(root, masterQuery->havingQual, true);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Right now this is not really needed, since we don't support correlated
|
|
||||||
* subqueries anyway. Once we do calling this is critical to do right after
|
|
||||||
* calling SS_process_sublinks, according to the postgres function comment.
|
|
||||||
*/
|
|
||||||
havingQual = SS_replace_correlation_vars(root, havingQual);
|
|
||||||
|
|
||||||
|
|
||||||
/* estimate aggregate execution costs */
|
|
||||||
memset(&aggregateCosts, 0, sizeof(AggClauseCosts));
|
|
||||||
get_agg_clause_costs(root, (Node *) aggregateTargetList, AGGSPLIT_SIMPLE,
|
|
||||||
&aggregateCosts);
|
|
||||||
|
|
||||||
get_agg_clause_costs(root, (Node *) havingQual, AGGSPLIT_SIMPLE, &aggregateCosts);
|
|
||||||
|
|
||||||
|
|
||||||
/* if we have grouping, then initialize appropriate information */
|
|
||||||
if (list_length(groupColumnList) > 0)
|
|
||||||
{
|
|
||||||
bool groupingIsHashable = grouping_is_hashable(groupColumnList);
|
|
||||||
bool groupingIsSortable = grouping_is_sortable(groupColumnList);
|
|
||||||
bool hasUnhashableAggregate = HasDistinctOrOrderByAggregate(masterQuery);
|
|
||||||
|
|
||||||
if (!groupingIsHashable && !groupingIsSortable)
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("grouped column list cannot be hashed or sorted")));
|
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Postgres hash aggregate strategy does not support distinct aggregates
|
|
||||||
* in group and order by with aggregate operations.
|
|
||||||
* see nodeAgg.c:build_pertrans_for_aggref(). In that case we use
|
|
||||||
* sorted agg strategy, otherwise we use hash strategy.
|
|
||||||
*
|
|
||||||
* If the master query contains hll aggregate functions and the client set
|
|
||||||
* hll.force_groupagg to on, then we choose to use group aggregation.
|
|
||||||
*/
|
|
||||||
if (!enable_hashagg || !groupingIsHashable || hasUnhashableAggregate ||
|
|
||||||
UseGroupAggregateWithHLL(masterQuery))
|
|
||||||
{
|
|
||||||
char *messageHint = NULL;
|
|
||||||
if (!enable_hashagg && groupingIsHashable)
|
|
||||||
{
|
|
||||||
messageHint = "Consider setting enable_hashagg to on.";
|
|
||||||
}
|
|
||||||
|
|
||||||
if (!groupingIsSortable)
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("grouped column list must cannot be sorted"),
|
|
||||||
errdetail("Having a distinct aggregate requires "
|
|
||||||
"grouped column list to be sortable."),
|
|
||||||
messageHint ? errhint("%s", messageHint) : 0));
|
|
||||||
}
|
|
||||||
|
|
||||||
aggregateStrategy = AGG_SORTED;
|
|
||||||
subPlan = (Plan *) make_sort_from_sortclauses(groupColumnList, subPlan);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
aggregateStrategy = AGG_HASHED;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
/* finally create the plan */
|
|
||||||
Agg *aggregatePlan = makeAggNode(groupColumnList, (List *) havingQual,
|
|
||||||
aggregateStrategy, aggregateTargetList, subPlan);
|
|
||||||
|
|
||||||
/* just for reproducible costs between different PostgreSQL versions */
|
|
||||||
aggregatePlan->plan.startup_cost = 0;
|
|
||||||
aggregatePlan->plan.total_cost = 0;
|
|
||||||
aggregatePlan->plan.plan_rows = 0;
|
|
||||||
|
|
||||||
return aggregatePlan;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* HasDistinctAggregate returns true if the query has a distinct
|
|
||||||
* aggregate in its target list or in having clause.
|
|
||||||
*/
|
*/
|
||||||
static bool
|
static bool
|
||||||
HasDistinctOrOrderByAggregate(Query *masterQuery)
|
FindCitusExtradataContainerRTE(Node *node, RangeTblEntry **result)
|
||||||
{
|
{
|
||||||
ListCell *allColumnCell = NULL;
|
if (node == NULL)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
List *targetVarList = pull_var_clause((Node *) masterQuery->targetList,
|
if (IsA(node, RangeTblEntry))
|
||||||
PVC_INCLUDE_AGGREGATES);
|
|
||||||
List *havingVarList = pull_var_clause(masterQuery->havingQual,
|
|
||||||
PVC_INCLUDE_AGGREGATES);
|
|
||||||
|
|
||||||
List *allColumnList = list_concat(targetVarList, havingVarList);
|
|
||||||
foreach(allColumnCell, allColumnList)
|
|
||||||
{
|
{
|
||||||
Node *columnNode = lfirst(allColumnCell);
|
RangeTblEntry *rangeTblEntry = castNode(RangeTblEntry, node);
|
||||||
if (IsA(columnNode, Aggref))
|
if (rangeTblEntry->rtekind == RTE_FUNCTION &&
|
||||||
|
list_length(rangeTblEntry->functions) == 1)
|
||||||
{
|
{
|
||||||
Aggref *aggref = (Aggref *) columnNode;
|
RangeTblFunction *rangeTblFunction = (RangeTblFunction *) linitial(
|
||||||
if (aggref->aggdistinct != NIL || aggref->aggorder != NIL)
|
rangeTblEntry->functions);
|
||||||
|
FuncExpr *funcExpr = castNode(FuncExpr, rangeTblFunction->funcexpr);
|
||||||
|
if (funcExpr->funcid == CitusExtraDataContainerFuncId())
|
||||||
{
|
{
|
||||||
|
*result = rangeTblEntry;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
/* query_tree_walker descends into RTEs */
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* UseGroupAggregateWithHLL first checks whether the HLL extension is loaded, if
|
|
||||||
* it is not then simply return false. Otherwise, checks whether the client set
|
|
||||||
* the hll.force_groupagg to on. If it is enabled and the master query contains
|
|
||||||
* hll aggregate function, it returns true.
|
|
||||||
*/
|
|
||||||
static bool
|
|
||||||
UseGroupAggregateWithHLL(Query *masterQuery)
|
|
||||||
{
|
|
||||||
Oid hllId = get_extension_oid(HLL_EXTENSION_NAME, true);
|
|
||||||
|
|
||||||
/* If HLL extension is not loaded, return false */
|
|
||||||
if (!OidIsValid(hllId))
|
|
||||||
{
|
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
else if (IsA(node, Query))
|
||||||
/* If HLL is loaded but related GUC is not set, return false */
|
|
||||||
const char *gucStrValue = GetConfigOption(HLL_FORCE_GROUPAGG_GUC_NAME, true, false);
|
|
||||||
if (gucStrValue == NULL || strcmp(gucStrValue, "off") == 0)
|
|
||||||
{
|
{
|
||||||
return false;
|
#if PG_VERSION_NUM >= 120000
|
||||||
}
|
const int flags = QTW_EXAMINE_RTES_BEFORE;
|
||||||
|
|
||||||
return QueryContainsAggregateWithHLL(masterQuery);
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* QueryContainsAggregateWithHLL returns true if the query has an hll aggregate
|
|
||||||
* function in it's target list.
|
|
||||||
*/
|
|
||||||
static bool
|
|
||||||
QueryContainsAggregateWithHLL(Query *query)
|
|
||||||
{
|
|
||||||
ListCell *varCell = NULL;
|
|
||||||
|
|
||||||
List *varList = pull_var_clause((Node *) query->targetList, PVC_INCLUDE_AGGREGATES);
|
|
||||||
foreach(varCell, varList)
|
|
||||||
{
|
|
||||||
Var *var = (Var *) lfirst(varCell);
|
|
||||||
if (nodeTag(var) == T_Aggref)
|
|
||||||
{
|
|
||||||
Aggref *aggref = (Aggref *) var;
|
|
||||||
int argCount = list_length(aggref->args);
|
|
||||||
Oid hllId = get_extension_oid(HLL_EXTENSION_NAME, false);
|
|
||||||
Oid hllSchemaOid = get_extension_schema(hllId);
|
|
||||||
const char *hllSchemaName = get_namespace_name(hllSchemaOid);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* If the obtained oid is InvalidOid for addFunctionId, that means
|
|
||||||
* we don't have an hll_add_agg function with the given argument count.
|
|
||||||
* So, we don't need to double check whether the obtained id is valid.
|
|
||||||
*/
|
|
||||||
Oid addFunctionId = FunctionOidExtended(hllSchemaName, HLL_ADD_AGGREGATE_NAME,
|
|
||||||
argCount, true);
|
|
||||||
Oid unionFunctionId = FunctionOid(hllSchemaName, HLL_UNION_AGGREGATE_NAME, 1);
|
|
||||||
|
|
||||||
if (aggref->aggfnoid == addFunctionId || aggref->aggfnoid == unionFunctionId)
|
|
||||||
{
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* BuildDistinctPlan creates an returns a plan for distinct. Depending on
|
|
||||||
* availability of hash function it chooses HashAgg over Sort/Unique
|
|
||||||
* plans.
|
|
||||||
* This function has a potential performance issue since we blindly set
|
|
||||||
* Plan nodes without looking at cost. We might need to revisit this
|
|
||||||
* if we have performance issues with select distinct queries.
|
|
||||||
*/
|
|
||||||
static Plan *
|
|
||||||
BuildDistinctPlan(Query *masterQuery, Plan *subPlan)
|
|
||||||
{
|
|
||||||
Plan *distinctPlan = NULL;
|
|
||||||
List *distinctClauseList = masterQuery->distinctClause;
|
|
||||||
List *targetList = copyObject(masterQuery->targetList);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* We don't need to add distinct plan if all of the columns used in group by
|
|
||||||
* clause also used in distinct clause, since group by clause guarantees the
|
|
||||||
* uniqueness of the target list for every row.
|
|
||||||
*/
|
|
||||||
if (IsGroupBySubsetOfDistinct(masterQuery->groupClause, masterQuery->distinctClause))
|
|
||||||
{
|
|
||||||
return subPlan;
|
|
||||||
}
|
|
||||||
|
|
||||||
Assert(masterQuery->distinctClause);
|
|
||||||
Assert(!masterQuery->hasDistinctOn);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* Create group by plan with HashAggregate if all distinct
|
|
||||||
* members are hashable, and not containing distinct aggregate.
|
|
||||||
* Otherwise create sort+unique plan.
|
|
||||||
*/
|
|
||||||
bool distinctClausesHashable = grouping_is_hashable(distinctClauseList);
|
|
||||||
bool hasUnhashableAggregate = HasDistinctOrOrderByAggregate(masterQuery);
|
|
||||||
|
|
||||||
if (enable_hashagg && distinctClausesHashable && !hasUnhashableAggregate)
|
|
||||||
{
|
|
||||||
distinctPlan = (Plan *) makeAggNode(distinctClauseList, NIL, AGG_HASHED,
|
|
||||||
targetList, subPlan);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
Sort *sortPlan = make_sort_from_sortclauses(masterQuery->distinctClause,
|
|
||||||
subPlan);
|
|
||||||
distinctPlan = (Plan *) make_unique_from_sortclauses((Plan *) sortPlan,
|
|
||||||
masterQuery->distinctClause);
|
|
||||||
}
|
|
||||||
|
|
||||||
return distinctPlan;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* makeAggNode creates a "Agg" plan node. groupClauseList is a list of
|
|
||||||
* SortGroupClause's.
|
|
||||||
*/
|
|
||||||
static Agg *
|
|
||||||
makeAggNode(List *groupClauseList, List *havingQual, AggStrategy aggrStrategy,
|
|
||||||
List *queryTargetList, Plan *subPlan)
|
|
||||||
{
|
|
||||||
Agg *aggNode = NULL;
|
|
||||||
int groupColumnCount = list_length(groupClauseList);
|
|
||||||
AttrNumber *groupColumnIdArray =
|
|
||||||
extract_grouping_cols(groupClauseList, subPlan->targetlist);
|
|
||||||
Oid *groupColumnOpArray = extract_grouping_ops(groupClauseList);
|
|
||||||
const int rowEstimate = 10;
|
|
||||||
|
|
||||||
#if (PG_VERSION_NUM >= 120000)
|
|
||||||
aggNode = make_agg(queryTargetList, havingQual, aggrStrategy,
|
|
||||||
AGGSPLIT_SIMPLE, groupColumnCount, groupColumnIdArray,
|
|
||||||
groupColumnOpArray,
|
|
||||||
extract_grouping_collations(groupClauseList,
|
|
||||||
subPlan->targetlist),
|
|
||||||
NIL, NIL, rowEstimate, subPlan);
|
|
||||||
#else
|
#else
|
||||||
aggNode = make_agg(queryTargetList, havingQual, aggrStrategy,
|
const int flags = QTW_EXAMINE_RTES;
|
||||||
AGGSPLIT_SIMPLE, groupColumnCount, groupColumnIdArray,
|
|
||||||
groupColumnOpArray,
|
|
||||||
NIL, NIL, rowEstimate, subPlan);
|
|
||||||
#endif
|
#endif
|
||||||
|
return query_tree_walker((Query *) node, FindCitusExtradataContainerRTE, result,
|
||||||
|
flags);
|
||||||
|
}
|
||||||
|
|
||||||
return aggNode;
|
return expression_tree_walker(node, FindCitusExtradataContainerRTE, result);
|
||||||
}
|
}
|
||||||
|
|
|
@ -113,7 +113,11 @@ static int ExtractRangeTableId(Node *node);
|
||||||
static void ExtractColumns(RangeTblEntry *rangeTableEntry, int rangeTableId,
|
static void ExtractColumns(RangeTblEntry *rangeTableEntry, int rangeTableId,
|
||||||
List *dependentJobList, List **columnNames, List **columnVars);
|
List *dependentJobList, List **columnNames, List **columnVars);
|
||||||
static RangeTblEntry * DerivedRangeTableEntry(MultiNode *multiNode, List *columnNames,
|
static RangeTblEntry * DerivedRangeTableEntry(MultiNode *multiNode, List *columnNames,
|
||||||
List *tableIdList);
|
List *tableIdList, List *funcColumnNames,
|
||||||
|
List *funcColumnTypes,
|
||||||
|
List *funcColumnTypeMods,
|
||||||
|
List *funcCollations);
|
||||||
|
|
||||||
static List * DerivedColumnNameList(uint32 columnCount, uint64 generatingJobId);
|
static List * DerivedColumnNameList(uint32 columnCount, uint64 generatingJobId);
|
||||||
static Query * BuildSubqueryJobQuery(MultiNode *multiNode);
|
static Query * BuildSubqueryJobQuery(MultiNode *multiNode);
|
||||||
static void UpdateAllColumnAttributes(Node *columnContainer, List *rangeTableList,
|
static void UpdateAllColumnAttributes(Node *columnContainer, List *rangeTableList,
|
||||||
|
@ -715,7 +719,8 @@ BuildJobQuery(MultiNode *multiNode, List *dependentJobList)
|
||||||
jobQuery->limitOffset = limitOffset;
|
jobQuery->limitOffset = limitOffset;
|
||||||
jobQuery->limitCount = limitCount;
|
jobQuery->limitCount = limitCount;
|
||||||
jobQuery->havingQual = havingQual;
|
jobQuery->havingQual = havingQual;
|
||||||
jobQuery->hasAggs = contain_agg_clause((Node *) targetList);
|
jobQuery->hasAggs = contain_agg_clause((Node *) targetList) ||
|
||||||
|
contain_agg_clause((Node *) havingQual);
|
||||||
jobQuery->distinctClause = distinctClause;
|
jobQuery->distinctClause = distinctClause;
|
||||||
jobQuery->hasDistinctOn = hasDistinctOn;
|
jobQuery->hasDistinctOn = hasDistinctOn;
|
||||||
|
|
||||||
|
@ -754,7 +759,8 @@ BuildReduceQuery(MultiExtendedOp *extendedOpNode, List *dependentJobList)
|
||||||
|
|
||||||
/* create a derived range table for the subtree below the collect */
|
/* create a derived range table for the subtree below the collect */
|
||||||
RangeTblEntry *rangeTableEntry = DerivedRangeTableEntry(multiNode, columnNameList,
|
RangeTblEntry *rangeTableEntry = DerivedRangeTableEntry(multiNode, columnNameList,
|
||||||
OutputTableIdList(multiNode));
|
OutputTableIdList(multiNode),
|
||||||
|
NIL, NIL, NIL, NIL);
|
||||||
rangeTableEntry->eref->colnames = columnNameList;
|
rangeTableEntry->eref->colnames = columnNameList;
|
||||||
ModifyRangeTblExtraData(rangeTableEntry, CITUS_RTE_SHARD, NULL, NULL, NULL);
|
ModifyRangeTblExtraData(rangeTableEntry, CITUS_RTE_SHARD, NULL, NULL, NULL);
|
||||||
derivedRangeTableList = lappend(derivedRangeTableList, rangeTableEntry);
|
derivedRangeTableList = lappend(derivedRangeTableList, rangeTableEntry);
|
||||||
|
@ -839,7 +845,8 @@ BaseRangeTableList(MultiNode *multiNode)
|
||||||
rangeTableEntry->alias = multiTable->alias;
|
rangeTableEntry->alias = multiTable->alias;
|
||||||
rangeTableEntry->relid = multiTable->relationId;
|
rangeTableEntry->relid = multiTable->relationId;
|
||||||
SetRangeTblExtraData(rangeTableEntry, CITUS_RTE_RELATION, NULL, NULL,
|
SetRangeTblExtraData(rangeTableEntry, CITUS_RTE_RELATION, NULL, NULL,
|
||||||
list_make1_int(multiTable->rangeTableId));
|
list_make1_int(multiTable->rangeTableId),
|
||||||
|
NIL, NIL, NIL, NIL);
|
||||||
|
|
||||||
baseRangeTableList = lappend(baseRangeTableList, rangeTableEntry);
|
baseRangeTableList = lappend(baseRangeTableList, rangeTableEntry);
|
||||||
}
|
}
|
||||||
|
@ -863,15 +870,18 @@ BaseRangeTableList(MultiNode *multiNode)
|
||||||
* on worker nodes in case of the master node query.
|
* on worker nodes in case of the master node query.
|
||||||
*/
|
*/
|
||||||
static RangeTblEntry *
|
static RangeTblEntry *
|
||||||
DerivedRangeTableEntry(MultiNode *multiNode, List *columnList, List *tableIdList)
|
DerivedRangeTableEntry(MultiNode *multiNode, List *columnList, List *tableIdList,
|
||||||
|
List *funcColumnNames, List *funcColumnTypes,
|
||||||
|
List *funcColumnTypeMods, List *funcCollations)
|
||||||
{
|
{
|
||||||
RangeTblEntry *rangeTableEntry = makeNode(RangeTblEntry);
|
RangeTblEntry *rangeTableEntry = makeNode(RangeTblEntry);
|
||||||
rangeTableEntry->inFromCl = true;
|
rangeTableEntry->inFromCl = true;
|
||||||
rangeTableEntry->eref = makeNode(Alias);
|
rangeTableEntry->eref = makeNode(Alias);
|
||||||
rangeTableEntry->eref->colnames = columnList;
|
rangeTableEntry->eref->colnames = columnList;
|
||||||
|
|
||||||
SetRangeTblExtraData(rangeTableEntry, CITUS_RTE_REMOTE_QUERY, NULL, NULL,
|
SetRangeTblExtraData(rangeTableEntry, CITUS_RTE_REMOTE_QUERY, NULL, NULL, tableIdList,
|
||||||
tableIdList);
|
funcColumnNames, funcColumnTypes, funcColumnTypeMods,
|
||||||
|
funcCollations);
|
||||||
|
|
||||||
return rangeTableEntry;
|
return rangeTableEntry;
|
||||||
}
|
}
|
||||||
|
@ -1221,9 +1231,36 @@ QueryJoinTree(MultiNode *multiNode, List *dependentJobList, List **rangeTableLis
|
||||||
List *columnNameList = DerivedColumnNameList(columnCount,
|
List *columnNameList = DerivedColumnNameList(columnCount,
|
||||||
dependentJob->jobId);
|
dependentJob->jobId);
|
||||||
|
|
||||||
|
List *funcColumnNames = NIL;
|
||||||
|
List *funcColumnTypes = NIL;
|
||||||
|
List *funcColumnTypeMods = NIL;
|
||||||
|
List *funcCollations = NIL;
|
||||||
|
|
||||||
|
TargetEntry *targetEntry = NULL;
|
||||||
|
foreach_ptr(targetEntry, dependentTargetList)
|
||||||
|
{
|
||||||
|
Node *expr = (Node *) targetEntry->expr;
|
||||||
|
|
||||||
|
char *name = targetEntry->resname;
|
||||||
|
if (name == NULL)
|
||||||
|
{
|
||||||
|
name = pstrdup("unnamed");
|
||||||
|
}
|
||||||
|
|
||||||
|
funcColumnNames = lappend(funcColumnNames, makeString(name));
|
||||||
|
|
||||||
|
funcColumnTypes = lappend_oid(funcColumnTypes, exprType(expr));
|
||||||
|
funcColumnTypeMods = lappend_int(funcColumnTypeMods, exprTypmod(expr));
|
||||||
|
funcCollations = lappend_oid(funcCollations, exprCollation(expr));
|
||||||
|
}
|
||||||
|
|
||||||
RangeTblEntry *rangeTableEntry = DerivedRangeTableEntry(multiNode,
|
RangeTblEntry *rangeTableEntry = DerivedRangeTableEntry(multiNode,
|
||||||
columnNameList,
|
columnNameList,
|
||||||
tableIdList);
|
tableIdList,
|
||||||
|
funcColumnNames,
|
||||||
|
funcColumnTypes,
|
||||||
|
funcColumnTypeMods,
|
||||||
|
funcCollations);
|
||||||
RangeTblRef *rangeTableRef = makeNode(RangeTblRef);
|
RangeTblRef *rangeTableRef = makeNode(RangeTblRef);
|
||||||
|
|
||||||
rangeTableRef->rtindex = list_length(*rangeTableList) + 1;
|
rangeTableRef->rtindex = list_length(*rangeTableList) + 1;
|
||||||
|
|
|
@ -45,6 +45,7 @@
|
||||||
#include "distributed/multi_join_order.h"
|
#include "distributed/multi_join_order.h"
|
||||||
#include "distributed/multi_logical_optimizer.h"
|
#include "distributed/multi_logical_optimizer.h"
|
||||||
#include "distributed/distributed_planner.h"
|
#include "distributed/distributed_planner.h"
|
||||||
|
#include "distributed/multi_master_planner.h"
|
||||||
#include "distributed/multi_router_planner.h"
|
#include "distributed/multi_router_planner.h"
|
||||||
#include "distributed/multi_server_executor.h"
|
#include "distributed/multi_server_executor.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
|
|
|
@ -2,3 +2,4 @@
|
||||||
|
|
||||||
/* bump version to 9.3-1 */
|
/* bump version to 9.3-1 */
|
||||||
|
|
||||||
|
#include "udfs/citus_extradata_container/9.3-1.sql"
|
||||||
|
|
|
@ -0,0 +1,12 @@
|
||||||
|
-- we use the citus_extradata_container function as a range table entry in the query part
|
||||||
|
-- executed on the coordinator. Now that we are letting this query be planned by the
|
||||||
|
-- postgres planner we need to be able to pass column names and type information with this
|
||||||
|
-- function. This requires the change of the prototype of the function and add a return
|
||||||
|
-- type. Changing the return type of the function requires we drop the function first.
|
||||||
|
DROP FUNCTION citus_extradata_container(INTERNAL);
|
||||||
|
CREATE OR REPLACE FUNCTION citus_extradata_container(INTERNAL)
|
||||||
|
RETURNS SETOF record
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME', $$citus_extradata_container$$;
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_extradata_container(INTERNAL)
|
||||||
|
IS 'placeholder function to store additional data in postgres node trees';
|
|
@ -0,0 +1,12 @@
|
||||||
|
-- we use the citus_extradata_container function as a range table entry in the query part
|
||||||
|
-- executed on the coordinator. Now that we are letting this query be planned by the
|
||||||
|
-- postgres planner we need to be able to pass column names and type information with this
|
||||||
|
-- function. This requires the change of the prototype of the function and add a return
|
||||||
|
-- type. Changing the return type of the function requires we drop the function first.
|
||||||
|
DROP FUNCTION citus_extradata_container(INTERNAL);
|
||||||
|
CREATE OR REPLACE FUNCTION citus_extradata_container(INTERNAL)
|
||||||
|
RETURNS SETOF record
|
||||||
|
LANGUAGE C
|
||||||
|
AS 'MODULE_PATHNAME', $$citus_extradata_container$$;
|
||||||
|
COMMENT ON FUNCTION pg_catalog.citus_extradata_container(INTERNAL)
|
||||||
|
IS 'placeholder function to store additional data in postgres node trees';
|
|
@ -72,9 +72,10 @@ PG_FUNCTION_INFO_V1(citus_extradata_container);
|
||||||
* will not be handled by out/readfuncs.c. For the current uses that's ok.
|
* will not be handled by out/readfuncs.c. For the current uses that's ok.
|
||||||
*/
|
*/
|
||||||
void
|
void
|
||||||
SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind, char *fragmentSchemaName,
|
||||||
char *fragmentSchemaName, char *fragmentTableName,
|
char *fragmentTableName, List *tableIdList, List *funcColumnNames,
|
||||||
List *tableIdList)
|
List *funcColumnTypes, List *funcColumnTypeMods,
|
||||||
|
List *funcCollations)
|
||||||
{
|
{
|
||||||
Assert(rte->eref);
|
Assert(rte->eref);
|
||||||
|
|
||||||
|
@ -127,6 +128,7 @@ SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
||||||
/* create function expression to store our faux arguments in */
|
/* create function expression to store our faux arguments in */
|
||||||
FuncExpr *fauxFuncExpr = makeNode(FuncExpr);
|
FuncExpr *fauxFuncExpr = makeNode(FuncExpr);
|
||||||
fauxFuncExpr->funcid = CitusExtraDataContainerFuncId();
|
fauxFuncExpr->funcid = CitusExtraDataContainerFuncId();
|
||||||
|
fauxFuncExpr->funcresulttype = RECORDOID;
|
||||||
fauxFuncExpr->funcretset = true;
|
fauxFuncExpr->funcretset = true;
|
||||||
fauxFuncExpr->location = -1;
|
fauxFuncExpr->location = -1;
|
||||||
fauxFuncExpr->args = list_make4(rteKindData, fragmentSchemaData,
|
fauxFuncExpr->args = list_make4(rteKindData, fragmentSchemaData,
|
||||||
|
@ -137,6 +139,10 @@ SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
||||||
|
|
||||||
/* set the column count to pass ruleutils checks, not used elsewhere */
|
/* set the column count to pass ruleutils checks, not used elsewhere */
|
||||||
fauxFunction->funccolcount = list_length(rte->eref->colnames);
|
fauxFunction->funccolcount = list_length(rte->eref->colnames);
|
||||||
|
fauxFunction->funccolnames = funcColumnNames;
|
||||||
|
fauxFunction->funccoltypes = funcColumnTypes;
|
||||||
|
fauxFunction->funccoltypmods = funcColumnTypeMods;
|
||||||
|
fauxFunction->funccolcollations = funcCollations;
|
||||||
|
|
||||||
rte->rtekind = RTE_FUNCTION;
|
rte->rtekind = RTE_FUNCTION;
|
||||||
rte->functions = list_make1(fauxFunction);
|
rte->functions = list_make1(fauxFunction);
|
||||||
|
@ -281,7 +287,7 @@ ModifyRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
||||||
|
|
||||||
SetRangeTblExtraData(rte, rteKind,
|
SetRangeTblExtraData(rte, rteKind,
|
||||||
fragmentSchemaName, fragmentTableName,
|
fragmentSchemaName, fragmentTableName,
|
||||||
tableIdList);
|
tableIdList, NIL, NIL, NIL, NIL);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -15,10 +15,13 @@
|
||||||
#include "executor/execdesc.h"
|
#include "executor/execdesc.h"
|
||||||
#include "nodes/plannodes.h"
|
#include "nodes/plannodes.h"
|
||||||
|
|
||||||
|
|
||||||
typedef struct CitusScanState
|
typedef struct CitusScanState
|
||||||
{
|
{
|
||||||
CustomScanState customScanState; /* underlying custom scan node */
|
CustomScanState customScanState; /* underlying custom scan node */
|
||||||
|
|
||||||
|
/* function that gets called before postgres starts its execution */
|
||||||
|
void (*PreExecScan)(struct CitusScanState *scanState);
|
||||||
|
|
||||||
DistributedPlan *distributedPlan; /* distributed execution plan */
|
DistributedPlan *distributedPlan; /* distributed execution plan */
|
||||||
MultiExecutorType executorType; /* distributed executor type */
|
MultiExecutorType executorType; /* distributed executor type */
|
||||||
bool finishedRemoteScan; /* flag to check if remote scan is finished */
|
bool finishedRemoteScan; /* flag to check if remote scan is finished */
|
||||||
|
|
|
@ -18,7 +18,9 @@
|
||||||
/* citus_nodefuncs.c */
|
/* citus_nodefuncs.c */
|
||||||
extern void SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
extern void SetRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
||||||
char *fragmentSchemaName, char *fragmentTableName,
|
char *fragmentSchemaName, char *fragmentTableName,
|
||||||
List *tableIdList);
|
List *tableIdList, List *funcColumnNames,
|
||||||
|
List *funcColumnTypes, List *funcColumnTypeMods,
|
||||||
|
List *funcCollations);
|
||||||
extern void ModifyRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
extern void ModifyRangeTblExtraData(RangeTblEntry *rte, CitusRTEKind rteKind,
|
||||||
char *fragmentSchemaName, char *fragmentTableName,
|
char *fragmentSchemaName, char *fragmentTableName,
|
||||||
List *tableIdList);
|
List *tableIdList);
|
||||||
|
|
|
@ -161,6 +161,26 @@ typedef struct DistributedPlanningContext
|
||||||
} DistributedPlanningContext;
|
} DistributedPlanningContext;
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CitusCustomScanPath is injected into the planner during the master query planning phase
|
||||||
|
* of the logical planner.
|
||||||
|
* We call out to the standard planner to plan the master query part for the output of the
|
||||||
|
* logical planner. This makes it easier to implement new sql features into the logical
|
||||||
|
* planner by not having to manually implement the plan creation for the query on the
|
||||||
|
* master.
|
||||||
|
*/
|
||||||
|
typedef struct CitusCustomScanPath
|
||||||
|
{
|
||||||
|
CustomPath custom_path;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Custom scan node computed by the citus planner that will produce the tuples for the
|
||||||
|
* path we are injecting during the planning of the master query
|
||||||
|
*/
|
||||||
|
CustomScan *remoteScan;
|
||||||
|
} CitusCustomScanPath;
|
||||||
|
|
||||||
|
|
||||||
extern PlannedStmt * distributed_planner(Query *parse, int cursorOptions,
|
extern PlannedStmt * distributed_planner(Query *parse, int cursorOptions,
|
||||||
ParamListInfo boundParams);
|
ParamListInfo boundParams);
|
||||||
extern List * ExtractRangeTableEntryList(Query *query);
|
extern List * ExtractRangeTableEntryList(Query *query);
|
||||||
|
|
|
@ -72,6 +72,7 @@ extern int ExecutorLevel;
|
||||||
extern void CitusExecutorStart(QueryDesc *queryDesc, int eflags);
|
extern void CitusExecutorStart(QueryDesc *queryDesc, int eflags);
|
||||||
extern void CitusExecutorRun(QueryDesc *queryDesc, ScanDirection direction, uint64 count,
|
extern void CitusExecutorRun(QueryDesc *queryDesc, ScanDirection direction, uint64 count,
|
||||||
bool execute_once);
|
bool execute_once);
|
||||||
|
extern void AdaptiveExecutorPreExecutorRun(CitusScanState *scanState);
|
||||||
extern TupleTableSlot * AdaptiveExecutor(CitusScanState *scanState);
|
extern TupleTableSlot * AdaptiveExecutor(CitusScanState *scanState);
|
||||||
extern uint64 ExecuteTaskListExtended(RowModifyLevel modLevel, List *taskList,
|
extern uint64 ExecuteTaskListExtended(RowModifyLevel modLevel, List *taskList,
|
||||||
TupleDesc tupleDescriptor,
|
TupleDesc tupleDescriptor,
|
||||||
|
@ -86,6 +87,7 @@ extern uint64 ExecuteTaskListIntoTupleStore(RowModifyLevel modLevel, List *taskL
|
||||||
extern void ExecuteUtilityTaskListWithoutResults(List *taskList);
|
extern void ExecuteUtilityTaskListWithoutResults(List *taskList);
|
||||||
extern uint64 ExecuteTaskList(RowModifyLevel modLevel, List *taskList, int
|
extern uint64 ExecuteTaskList(RowModifyLevel modLevel, List *taskList, int
|
||||||
targetPoolSize);
|
targetPoolSize);
|
||||||
|
extern bool IsCitusCustomState(PlanState *planState);
|
||||||
extern TupleTableSlot * CitusExecScan(CustomScanState *node);
|
extern TupleTableSlot * CitusExecScan(CustomScanState *node);
|
||||||
extern TupleTableSlot * ReturnTupleFromTuplestore(CitusScanState *scanState);
|
extern TupleTableSlot * ReturnTupleFromTuplestore(CitusScanState *scanState);
|
||||||
extern void LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob);
|
extern void LoadTuplesIntoTupleStore(CitusScanState *citusScanState, Job *workerJob);
|
||||||
|
|
|
@ -192,6 +192,7 @@ extern bool TargetListOnPartitionColumn(Query *query, List *targetEntryList);
|
||||||
extern bool FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *));
|
extern bool FindNodeCheckInRangeTableList(List *rtable, bool (*check)(Node *));
|
||||||
extern bool IsDistributedTableRTE(Node *node);
|
extern bool IsDistributedTableRTE(Node *node);
|
||||||
extern bool QueryContainsDistributedTableRTE(Query *query);
|
extern bool QueryContainsDistributedTableRTE(Query *query);
|
||||||
|
extern bool IsCitusExtraDataContainerRelation(RangeTblEntry *rte);
|
||||||
extern bool ContainsReadIntermediateResultFunction(Node *node);
|
extern bool ContainsReadIntermediateResultFunction(Node *node);
|
||||||
extern bool ContainsReadIntermediateResultArrayFunction(Node *node);
|
extern bool ContainsReadIntermediateResultArrayFunction(Node *node);
|
||||||
extern char * FindIntermediateResultIdIfExists(RangeTblEntry *rte);
|
extern char * FindIntermediateResultIdIfExists(RangeTblEntry *rte);
|
||||||
|
|
|
@ -18,13 +18,23 @@
|
||||||
#include "nodes/parsenodes.h"
|
#include "nodes/parsenodes.h"
|
||||||
#include "nodes/plannodes.h"
|
#include "nodes/plannodes.h"
|
||||||
|
|
||||||
|
#if PG_VERSION_NUM >= 120000
|
||||||
|
#include "nodes/pathnodes.h"
|
||||||
|
#else
|
||||||
|
#include "nodes/relation.h"
|
||||||
|
#endif
|
||||||
|
|
||||||
|
|
||||||
/* Function declarations for building local plans on the master node */
|
/* Function declarations for building local plans on the master node */
|
||||||
struct DistributedPlan;
|
struct DistributedPlan;
|
||||||
struct CustomScan;
|
struct CustomScan;
|
||||||
|
extern Path * CreateCitusCustomScanPath(PlannerInfo *root, RelOptInfo *relOptInfo,
|
||||||
|
Index restrictionIndex, RangeTblEntry *rte,
|
||||||
|
CustomScan *remoteScan);
|
||||||
extern PlannedStmt * MasterNodeSelectPlan(struct DistributedPlan *distributedPlan,
|
extern PlannedStmt * MasterNodeSelectPlan(struct DistributedPlan *distributedPlan,
|
||||||
struct CustomScan *dataScan);
|
struct CustomScan *dataScan);
|
||||||
extern Unique * make_unique_from_sortclauses(Plan *lefttree, List *distinctList);
|
extern Unique * make_unique_from_sortclauses(Plan *lefttree, List *distinctList);
|
||||||
|
extern bool ReplaceCitusExtraDataContainer;
|
||||||
|
extern CustomScan *ReplaceCitusExtraDataContainerWithCustomScan;
|
||||||
|
|
||||||
#endif /* MULTI_MASTER_PLANNER_H */
|
#endif /* MULTI_MASTER_PLANNER_H */
|
||||||
|
|
|
@ -1,6 +1,6 @@
|
||||||
# Rules to normalize test outputs. Our custom diff tool passes test output
|
# Rules to normalize test outputs. Our custom diff tool passes test output
|
||||||
# of tests in normalized_tests.lst through the substitution rules in this file
|
# of tests through the substitution rules in this file before doing the
|
||||||
# before doing the actual comparison.
|
# actual comparison.
|
||||||
#
|
#
|
||||||
# An example of when this is useful is when an error happens on a different
|
# An example of when this is useful is when an error happens on a different
|
||||||
# port number, or a different worker shard, or a different placement, etc.
|
# port number, or a different worker shard, or a different placement, etc.
|
||||||
|
@ -92,3 +92,6 @@ s/read_intermediate_result\('insert_select_[0-9]+_/read_intermediate_result('ins
|
||||||
# ignore job id in repartitioned insert/select
|
# ignore job id in repartitioned insert/select
|
||||||
s/repartitioned_results_[0-9]+/repartitioned_results_xxxxx/g
|
s/repartitioned_results_[0-9]+/repartitioned_results_xxxxx/g
|
||||||
|
|
||||||
|
# ignore first parameter for citus_extradata_container due to differences between pg11 and pg12
|
||||||
|
# can be removed when we remove PG_VERSION_NUM >= 120000
|
||||||
|
s/pg_catalog.citus_extradata_container\([0-9]+/pg_catalog.citus_extradata_container\(XXX/g
|
||||||
|
|
|
@ -80,7 +80,8 @@ SELECT create_reference_table('ref_table');
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- single hash repartition after bcast joins
|
-- single hash repartition after bcast joins
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
count(*)
|
count(*)
|
||||||
FROM
|
FROM
|
||||||
ref_table r1, single_hash_repartition_second t1, single_hash_repartition_first t2
|
ref_table r1, single_hash_repartition_second t1, single_hash_repartition_first t2
|
||||||
|
@ -88,8 +89,8 @@ WHERE
|
||||||
r1.id = t1.id AND t2.sum = t1.id;
|
r1.id = t1.id AND t2.sum = t1.id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: None, not supported for re-partition queries
|
Tasks Shown: None, not supported for re-partition queries
|
||||||
-> MapMergeJob
|
-> MapMergeJob
|
||||||
|
@ -98,7 +99,8 @@ WHERE
|
||||||
(7 rows)
|
(7 rows)
|
||||||
|
|
||||||
-- a more complicated join order, first colocated join, later single hash repartition join
|
-- a more complicated join order, first colocated join, later single hash repartition join
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
count(*)
|
count(*)
|
||||||
FROM
|
FROM
|
||||||
single_hash_repartition_first t1, single_hash_repartition_first t2, single_hash_repartition_second t3
|
single_hash_repartition_first t1, single_hash_repartition_first t2, single_hash_repartition_second t3
|
||||||
|
@ -106,8 +108,8 @@ WHERE
|
||||||
t1.id = t2.id AND t1.sum = t3.id;
|
t1.id = t2.id AND t1.sum = t3.id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: None, not supported for re-partition queries
|
Tasks Shown: None, not supported for re-partition queries
|
||||||
-> MapMergeJob
|
-> MapMergeJob
|
||||||
|
|
|
@ -22,12 +22,12 @@ order by s_i_id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Sort
|
||||||
Sort Key: s_i_id
|
Sort Key: remote_scan.s_i_id
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Function Scan on read_intermediate_result intermediate_result
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: ((pg_catalog.sum(worker_column_3))::bigint > $0)
|
Filter: ((pg_catalog.sum(remote_scan.worker_column_3))::bigint > $0)
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
-> Distributed Subplan XXX_1
|
-> Distributed Subplan XXX_1
|
||||||
-> Aggregate
|
-> Aggregate
|
||||||
|
@ -68,12 +68,12 @@ order by s_i_id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Sort
|
||||||
Sort Key: s_i_id
|
Sort Key: remote_scan.s_i_id
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Function Scan on read_intermediate_result intermediate_result
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: ((pg_catalog.sum(worker_column_3))::bigint > $0)
|
Filter: ((pg_catalog.sum(remote_scan.worker_column_3))::bigint > $0)
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
-> Distributed Subplan XXX_1
|
-> Distributed Subplan XXX_1
|
||||||
-> Aggregate
|
-> Aggregate
|
||||||
|
@ -101,8 +101,8 @@ having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from st
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
HashAggregate
|
HashAggregate
|
||||||
Group Key: s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: ((pg_catalog.sum(worker_column_3))::bigint > $0)
|
Filter: ((pg_catalog.sum(remote_scan.worker_column_3))::bigint > $0)
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Function Scan on read_intermediate_result intermediate_result
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -124,50 +124,56 @@ having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from st
|
||||||
-> Seq Scan on stock_1640000 stock
|
-> Seq Scan on stock_1640000 stock
|
||||||
(22 rows)
|
(22 rows)
|
||||||
|
|
||||||
explain select s_i_id, sum(s_order_cnt) as ordercount
|
explain (costs false)
|
||||||
|
select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
from stock s
|
from stock s
|
||||||
group by s_i_id
|
group by s_i_id
|
||||||
having (select true)
|
having (select true)
|
||||||
order by s_i_id;
|
order by s_i_id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort
|
||||||
Sort Key: remote_scan.s_i_id
|
Sort Key: remote_scan.s_i_id
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Result (cost=0.00..0.01 rows=1 width=1)
|
-> Result
|
||||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
-> HashAggregate
|
||||||
Group Key: remote_scan.s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
|
-> Result
|
||||||
|
One-Time Filter: $0
|
||||||
|
-> Custom Scan (Citus Adaptive)
|
||||||
Filter: $0
|
Filter: $0
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: One of 4
|
Tasks Shown: One of 4
|
||||||
-> Task
|
-> Task
|
||||||
Node: host=localhost port=xxxxx dbname=regression
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
-> HashAggregate (cost=40.60..42.60 rows=200 width=12)
|
-> HashAggregate
|
||||||
Group Key: s.s_i_id
|
Group Key: s.s_i_id
|
||||||
-> Seq Scan on stock_1640000 s (cost=0.00..30.40 rows=2040 width=8)
|
-> Seq Scan on stock_1640000 s
|
||||||
(15 rows)
|
(17 rows)
|
||||||
|
|
||||||
explain select s_i_id, sum(s_order_cnt) as ordercount
|
explain (costs false)
|
||||||
|
select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
from stock s
|
from stock s
|
||||||
group by s_i_id
|
group by s_i_id
|
||||||
having (select true);
|
having (select true);
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
HashAggregate
|
||||||
Group Key: remote_scan.s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: $0
|
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Result (cost=0.00..0.01 rows=1 width=1)
|
-> Result
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Result
|
||||||
|
One-Time Filter: $0
|
||||||
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
Filter: $0
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: One of 4
|
Tasks Shown: One of 4
|
||||||
-> Task
|
-> Task
|
||||||
Node: host=localhost port=xxxxx dbname=regression
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
-> HashAggregate (cost=40.60..42.60 rows=200 width=12)
|
-> HashAggregate
|
||||||
Group Key: s.s_i_id
|
Group Key: s.s_i_id
|
||||||
-> Seq Scan on stock_1640000 s (cost=0.00..30.40 rows=2040 width=8)
|
-> Seq Scan on stock_1640000 s
|
||||||
(13 rows)
|
(15 rows)
|
||||||
|
|
||||||
select s_i_id, sum(s_order_cnt) as ordercount
|
select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
from stock
|
from stock
|
||||||
|
|
|
@ -27,12 +27,12 @@ order by s_i_id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Sort
|
||||||
Sort Key: s_i_id
|
Sort Key: remote_scan.s_i_id
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Function Scan on read_intermediate_result intermediate_result
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: ((pg_catalog.sum(worker_column_3))::bigint > $0)
|
Filter: ((pg_catalog.sum(remote_scan.worker_column_3))::bigint > $0)
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
-> Distributed Subplan XXX_1
|
-> Distributed Subplan XXX_1
|
||||||
-> Aggregate
|
-> Aggregate
|
||||||
|
@ -73,12 +73,12 @@ order by s_i_id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Sort
|
||||||
Sort Key: s_i_id
|
Sort Key: remote_scan.s_i_id
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Function Scan on read_intermediate_result intermediate_result
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: ((pg_catalog.sum(worker_column_3))::bigint > $0)
|
Filter: ((pg_catalog.sum(remote_scan.worker_column_3))::bigint > $0)
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
-> Distributed Subplan XXX_1
|
-> Distributed Subplan XXX_1
|
||||||
-> Aggregate
|
-> Aggregate
|
||||||
|
@ -106,8 +106,8 @@ having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from st
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
HashAggregate
|
HashAggregate
|
||||||
Group Key: s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
Filter: ((pg_catalog.sum(worker_column_3))::bigint > $0)
|
Filter: ((pg_catalog.sum(remote_scan.worker_column_3))::bigint > $0)
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Function Scan on read_intermediate_result intermediate_result
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -136,14 +136,39 @@ having (select true)
|
||||||
order by s_i_id;
|
order by s_i_id;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort (cost=510.65..511.15 rows=200 width=12)
|
||||||
Sort Key: remote_scan.s_i_id
|
Sort Key: remote_scan.s_i_id
|
||||||
InitPlan 1 (returns $0)
|
InitPlan 1 (returns $0)
|
||||||
-> Result (cost=0.00..0.01 rows=1 width=1)
|
-> Result (cost=0.00..0.01 rows=1 width=1)
|
||||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
-> HashAggregate (cost=500.00..503.00 rows=200 width=12)
|
||||||
Group Key: remote_scan.s_i_id
|
Group Key: remote_scan.s_i_id
|
||||||
|
-> Result (cost=0.00..0.00 rows=100000 width=12)
|
||||||
|
One-Time Filter: $0
|
||||||
|
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=100000 width=12)
|
||||||
|
Filter: $0
|
||||||
|
Task Count: 4
|
||||||
|
Tasks Shown: One of 4
|
||||||
|
-> Task
|
||||||
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
|
-> HashAggregate (cost=40.60..42.60 rows=200 width=12)
|
||||||
|
Group Key: s.s_i_id
|
||||||
|
-> Seq Scan on stock_1640000 s (cost=0.00..30.40 rows=2040 width=8)
|
||||||
|
(17 rows)
|
||||||
|
|
||||||
|
explain select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
|
from stock s
|
||||||
|
group by s_i_id
|
||||||
|
having (select true);
|
||||||
|
QUERY PLAN
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
HashAggregate (cost=500.01..503.01 rows=200 width=12)
|
||||||
|
Group Key: remote_scan.s_i_id
|
||||||
|
InitPlan 1 (returns $0)
|
||||||
|
-> Result (cost=0.00..0.01 rows=1 width=1)
|
||||||
|
-> Result (cost=0.00..0.00 rows=100000 width=12)
|
||||||
|
One-Time Filter: $0
|
||||||
|
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=100000 width=12)
|
||||||
Filter: $0
|
Filter: $0
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: One of 4
|
Tasks Shown: One of 4
|
||||||
-> Task
|
-> Task
|
||||||
|
@ -153,27 +178,6 @@ order by s_i_id;
|
||||||
-> Seq Scan on stock_1640000 s (cost=0.00..30.40 rows=2040 width=8)
|
-> Seq Scan on stock_1640000 s (cost=0.00..30.40 rows=2040 width=8)
|
||||||
(15 rows)
|
(15 rows)
|
||||||
|
|
||||||
explain select s_i_id, sum(s_order_cnt) as ordercount
|
|
||||||
from stock s
|
|
||||||
group by s_i_id
|
|
||||||
having (select true);
|
|
||||||
QUERY PLAN
|
|
||||||
---------------------------------------------------------------------
|
|
||||||
HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
|
||||||
Group Key: remote_scan.s_i_id
|
|
||||||
Filter: $0
|
|
||||||
InitPlan 1 (returns $0)
|
|
||||||
-> Result (cost=0.00..0.01 rows=1 width=1)
|
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
|
||||||
Task Count: 4
|
|
||||||
Tasks Shown: One of 4
|
|
||||||
-> Task
|
|
||||||
Node: host=localhost port=xxxxx dbname=regression
|
|
||||||
-> HashAggregate (cost=40.60..42.60 rows=200 width=12)
|
|
||||||
Group Key: s.s_i_id
|
|
||||||
-> Seq Scan on stock_1640000 s (cost=0.00..30.40 rows=2040 width=8)
|
|
||||||
(13 rows)
|
|
||||||
|
|
||||||
select s_i_id, sum(s_order_cnt) as ordercount
|
select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
from stock
|
from stock
|
||||||
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
|
where s_order_cnt > (select sum(s_order_cnt) * .005 as where_query from stock)
|
||||||
|
|
|
@ -988,19 +988,19 @@ GROUP BY
|
||||||
key
|
key
|
||||||
HAVING
|
HAVING
|
||||||
(count(*) > (SELECT max FROM cte_1))
|
(count(*) > (SELECT max FROM cte_1))
|
||||||
ORDER BY 2 DESC
|
ORDER BY 2 DESC, 1 DESC
|
||||||
LIMIT 5;
|
LIMIT 5;
|
||||||
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
||||||
DEBUG: Router planner cannot handle multi-shard select queries
|
DEBUG: Router planner cannot handle multi-shard select queries
|
||||||
DEBUG: Router planner cannot handle multi-shard select queries
|
DEBUG: Router planner cannot handle multi-shard select queries
|
||||||
DEBUG: generating subplan XXX_1 for subquery SELECT max(key) AS max FROM cte_inline.test_table
|
DEBUG: generating subplan XXX_1 for subquery SELECT max(key) AS max FROM cte_inline.test_table
|
||||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, count(*) AS count FROM cte_inline.test_table GROUP BY key HAVING (count(*) OPERATOR(pg_catalog.>) (SELECT cte_1.max FROM (SELECT intermediate_result.max FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(max integer)) cte_1)) ORDER BY (count(*)) DESC LIMIT 5
|
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, count(*) AS count FROM cte_inline.test_table GROUP BY key HAVING (count(*) OPERATOR(pg_catalog.>) (SELECT cte_1.max FROM (SELECT intermediate_result.max FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(max integer)) cte_1)) ORDER BY (count(*)) DESC, key DESC LIMIT 5
|
||||||
DEBUG: Router planner cannot handle multi-shard select queries
|
DEBUG: Router planner cannot handle multi-shard select queries
|
||||||
key | count
|
key | count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
8 | 40
|
||||||
5 | 40
|
5 | 40
|
||||||
2 | 40
|
2 | 40
|
||||||
8 | 40
|
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- cte used in ORDER BY just works fine
|
-- cte used in ORDER BY just works fine
|
||||||
|
|
|
@ -854,21 +854,21 @@ GROUP BY
|
||||||
key
|
key
|
||||||
HAVING
|
HAVING
|
||||||
(count(*) > (SELECT max FROM cte_1))
|
(count(*) > (SELECT max FROM cte_1))
|
||||||
ORDER BY 2 DESC
|
ORDER BY 2 DESC, 1 DESC
|
||||||
LIMIT 5;
|
LIMIT 5;
|
||||||
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
DEBUG: CTE cte_1 is going to be inlined via distributed planning
|
||||||
DEBUG: Router planner cannot handle multi-shard select queries
|
DEBUG: Router planner cannot handle multi-shard select queries
|
||||||
DEBUG: Router planner cannot handle multi-shard select queries
|
DEBUG: Router planner cannot handle multi-shard select queries
|
||||||
DEBUG: generating subplan XXX_1 for subquery SELECT max(key) AS max FROM cte_inline.test_table
|
DEBUG: generating subplan XXX_1 for subquery SELECT max(key) AS max FROM cte_inline.test_table
|
||||||
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, count(*) AS count FROM cte_inline.test_table GROUP BY key HAVING (count(*) OPERATOR(pg_catalog.>) (SELECT cte_1.max FROM (SELECT intermediate_result.max FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(max integer)) cte_1)) ORDER BY (count(*)) DESC LIMIT 5
|
DEBUG: Plan XXX query after replacing subqueries and CTEs: SELECT key, count(*) AS count FROM cte_inline.test_table GROUP BY key HAVING (count(*) OPERATOR(pg_catalog.>) (SELECT cte_1.max FROM (SELECT intermediate_result.max FROM read_intermediate_result('XXX_1'::text, 'binary'::citus_copy_format) intermediate_result(max integer)) cte_1)) ORDER BY (count(*)) DESC, key DESC LIMIT 5
|
||||||
DEBUG: Router planner cannot handle multi-shard select queries
|
DEBUG: Router planner cannot handle multi-shard select queries
|
||||||
key | count
|
key | count
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
0 | 44
|
0 | 44
|
||||||
9 | 40
|
9 | 40
|
||||||
5 | 40
|
|
||||||
8 | 40
|
8 | 40
|
||||||
6 | 40
|
6 | 40
|
||||||
|
5 | 40
|
||||||
(5 rows)
|
(5 rows)
|
||||||
|
|
||||||
-- cte used in ORDER BY just works fine
|
-- cte used in ORDER BY just works fine
|
||||||
|
|
|
@ -0,0 +1,60 @@
|
||||||
|
-- Testing a having clause that could have been a where clause between a distributed table
|
||||||
|
-- and a reference table. This query was the cause for intermediate results not being
|
||||||
|
-- available during the replace of the planner for the master query with the standard
|
||||||
|
-- planner.
|
||||||
|
-- Since the having clause could have been a where clause the having clause on the grouping
|
||||||
|
-- on the coordinator is replaced with a Result node containing a One-time filter if the
|
||||||
|
-- having qual (one-time filter works because the query doesn't change with the tuples
|
||||||
|
-- returned from below).
|
||||||
|
SELECT count(*),
|
||||||
|
o_orderstatus
|
||||||
|
FROM orders
|
||||||
|
GROUP BY 2
|
||||||
|
HAVING (
|
||||||
|
SELECT count(*)
|
||||||
|
FROM customer
|
||||||
|
) > 0;
|
||||||
|
count | o_orderstatus
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
1461 | O
|
||||||
|
75 | P
|
||||||
|
1449 | F
|
||||||
|
(3 rows)
|
||||||
|
|
||||||
|
-- lets pin the plan in the test as well
|
||||||
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*),
|
||||||
|
o_orderstatus
|
||||||
|
FROM orders
|
||||||
|
GROUP BY 2
|
||||||
|
HAVING (
|
||||||
|
SELECT count(*)
|
||||||
|
FROM customer
|
||||||
|
) > 0;
|
||||||
|
QUERY PLAN
|
||||||
|
---------------------------------------------------------------------
|
||||||
|
HashAggregate
|
||||||
|
Group Key: remote_scan.o_orderstatus
|
||||||
|
InitPlan 1 (returns $0)
|
||||||
|
-> Function Scan on read_intermediate_result intermediate_result
|
||||||
|
-> Result
|
||||||
|
One-Time Filter: ($0 > 0)
|
||||||
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
Filter: ($0 > 0)
|
||||||
|
-> Distributed Subplan XXX_1
|
||||||
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
Task Count: 1
|
||||||
|
Tasks Shown: All
|
||||||
|
-> Task
|
||||||
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
|
-> Aggregate
|
||||||
|
-> Seq Scan on customer_360001 customer
|
||||||
|
Task Count: 2
|
||||||
|
Tasks Shown: One of 2
|
||||||
|
-> Task
|
||||||
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
|
-> HashAggregate
|
||||||
|
Group Key: orders.o_orderstatus
|
||||||
|
-> Seq Scan on orders_290002 orders
|
||||||
|
(23 rows)
|
||||||
|
|
|
@ -563,7 +563,7 @@ EXPLAIN INSERT INTO target_table SELECT a, max(b) FROM source_table GROUP BY a;
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Custom Scan (Citus INSERT ... SELECT) (cost=0.00..0.00 rows=0 width=0)
|
Custom Scan (Citus INSERT ... SELECT) (cost=0.00..0.00 rows=0 width=0)
|
||||||
INSERT/SELECT method: repartition
|
INSERT/SELECT method: repartition
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=100000 width=8)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: One of 4
|
Tasks Shown: One of 4
|
||||||
-> Task
|
-> Task
|
||||||
|
|
|
@ -747,8 +747,9 @@ DEBUG: Subplan XXX_1 will be sent to localhost:xxxxx
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
-- test with INSERT SELECT via coordinator
|
-- test with INSERT SELECT via coordinator
|
||||||
-- INSERT .. SELECT via coordinator that doesn't have any intermediate results
|
-- INSERT .. SELECT via coordinator that doesn't have any intermediate results
|
||||||
|
-- We use offset 1 to make sure the result needs to be pulled to the coordinator, offset 0 would be optimized away
|
||||||
INSERT INTO table_1
|
INSERT INTO table_1
|
||||||
SELECT * FROM table_2 OFFSET 0;
|
SELECT * FROM table_2 OFFSET 1;
|
||||||
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
|
DEBUG: OFFSET clauses are not allowed in distributed INSERT ... SELECT queries
|
||||||
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
DEBUG: Collecting INSERT ... SELECT results on coordinator
|
||||||
-- INSERT .. SELECT via coordinator which has intermediate result,
|
-- INSERT .. SELECT via coordinator which has intermediate result,
|
||||||
|
|
|
@ -1147,7 +1147,7 @@ Aggregate
|
||||||
RESET citus.task_executor_type;
|
RESET citus.task_executor_type;
|
||||||
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
|
PREPARE router_executor_query AS SELECT l_quantity FROM lineitem WHERE l_orderkey = 5;
|
||||||
EXPLAIN EXECUTE router_executor_query;
|
EXPLAIN EXECUTE router_executor_query;
|
||||||
Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=100000 width=18)
|
||||||
Task Count: 1
|
Task Count: 1
|
||||||
Tasks Shown: All
|
Tasks Shown: All
|
||||||
-> Task
|
-> Task
|
||||||
|
@ -1170,7 +1170,7 @@ Aggregate
|
||||||
-- at least make sure to fail without crashing
|
-- at least make sure to fail without crashing
|
||||||
PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1;
|
PREPARE router_executor_query_param(int) AS SELECT l_quantity FROM lineitem WHERE l_orderkey = $1;
|
||||||
EXPLAIN EXECUTE router_executor_query_param(5);
|
EXPLAIN EXECUTE router_executor_query_param(5);
|
||||||
Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=100000 width=18)
|
||||||
Task Count: 1
|
Task Count: 1
|
||||||
Tasks Shown: All
|
Tasks Shown: All
|
||||||
-> Task
|
-> Task
|
||||||
|
|
|
@ -70,7 +70,8 @@ SELECT create_distributed_table('customer_hash', 'c_custkey');
|
||||||
|
|
||||||
SET client_min_messages TO DEBUG2;
|
SET client_min_messages TO DEBUG2;
|
||||||
-- The following query checks that we can correctly handle self-joins
|
-- The following query checks that we can correctly handle self-joins
|
||||||
EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT l1.l_quantity FROM lineitem l1, lineitem l2
|
||||||
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
|
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
LOG: join order: [ "lineitem" ][ local partition join "lineitem" ]
|
LOG: join order: [ "lineitem" ][ local partition join "lineitem" ]
|
||||||
|
@ -78,7 +79,7 @@ DEBUG: join prunable for intervals [1,5986] and [8997,14947]
|
||||||
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
|
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(2 rows)
|
(2 rows)
|
||||||
|
|
||||||
|
@ -86,92 +87,101 @@ SET client_min_messages TO LOG;
|
||||||
-- The following queries check that we correctly handle joins and OR clauses. In
|
-- The following queries check that we correctly handle joins and OR clauses. In
|
||||||
-- particular, these queries check that we factorize out OR clauses if possible,
|
-- particular, these queries check that we factorize out OR clauses if possible,
|
||||||
-- and that we default to a cartesian product otherwise.
|
-- and that we default to a cartesian product otherwise.
|
||||||
EXPLAIN SELECT count(*) FROM lineitem, orders
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM lineitem, orders
|
||||||
WHERE (l_orderkey = o_orderkey AND l_quantity > 5)
|
WHERE (l_orderkey = o_orderkey AND l_quantity > 5)
|
||||||
OR (l_orderkey = o_orderkey AND l_quantity < 10);
|
OR (l_orderkey = o_orderkey AND l_quantity < 10);
|
||||||
LOG: join order: [ "lineitem" ][ local partition join "orders" ]
|
LOG: join order: [ "lineitem" ][ local partition join "orders" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
EXPLAIN SELECT l_quantity FROM lineitem, orders
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT l_quantity FROM lineitem, orders
|
||||||
WHERE (l_orderkey = o_orderkey OR l_quantity > 5);
|
WHERE (l_orderkey = o_orderkey OR l_quantity > 5);
|
||||||
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
ERROR: complex joins are only supported when all distributed tables are joined on their distribution columns with equal operator
|
||||||
EXPLAIN SELECT count(*) FROM orders, lineitem_hash
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders, lineitem_hash
|
||||||
WHERE o_orderkey = l_orderkey;
|
WHERE o_orderkey = l_orderkey;
|
||||||
LOG: join order: [ "orders" ][ single range partition join "lineitem_hash" ]
|
LOG: join order: [ "orders" ][ single range partition join "lineitem_hash" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Verify we handle local joins between two hash-partitioned tables.
|
-- Verify we handle local joins between two hash-partitioned tables.
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash, lineitem_hash
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders_hash, lineitem_hash
|
||||||
WHERE o_orderkey = l_orderkey;
|
WHERE o_orderkey = l_orderkey;
|
||||||
LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ]
|
LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Validate that we can handle broadcast joins with hash-partitioned tables.
|
-- Validate that we can handle broadcast joins with hash-partitioned tables.
|
||||||
EXPLAIN SELECT count(*) FROM customer_hash, nation
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM customer_hash, nation
|
||||||
WHERE c_nationkey = n_nationkey;
|
WHERE c_nationkey = n_nationkey;
|
||||||
LOG: join order: [ "customer_hash" ][ reference join "nation" ]
|
LOG: join order: [ "customer_hash" ][ reference join "nation" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Validate that we don't use a single-partition join method for a hash
|
-- Validate that we don't use a single-partition join method for a hash
|
||||||
-- re-partitioned table, thus preventing a partition of just the customer table.
|
-- re-partitioned table, thus preventing a partition of just the customer table.
|
||||||
EXPLAIN SELECT count(*) FROM orders, lineitem, customer_append
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders, lineitem, customer_append
|
||||||
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
|
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
|
||||||
LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer_append" ]
|
LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer_append" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Validate that we don't chose a single-partition join method with a
|
-- Validate that we don't chose a single-partition join method with a
|
||||||
-- hash-partitioned base table
|
-- hash-partitioned base table
|
||||||
EXPLAIN SELECT count(*) FROM orders, customer_hash
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders, customer_hash
|
||||||
WHERE c_custkey = o_custkey;
|
WHERE c_custkey = o_custkey;
|
||||||
LOG: join order: [ "orders" ][ dual partition join "customer_hash" ]
|
LOG: join order: [ "orders" ][ dual partition join "customer_hash" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Validate that we can re-partition a hash partitioned table to join with a
|
-- Validate that we can re-partition a hash partitioned table to join with a
|
||||||
-- range partitioned one.
|
-- range partitioned one.
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash, customer_append
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders_hash, customer_append
|
||||||
WHERE c_custkey = o_custkey;
|
WHERE c_custkey = o_custkey;
|
||||||
LOG: join order: [ "orders_hash" ][ single range partition join "customer_append" ]
|
LOG: join order: [ "orders_hash" ][ single range partition join "customer_append" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Validate a 4 way join that could be done locally is planned as such by the logical
|
-- Validate a 4 way join that could be done locally is planned as such by the logical
|
||||||
-- planner. It used to be planned as a repartition join due to no 1 table being directly
|
-- planner. It used to be planned as a repartition join due to no 1 table being directly
|
||||||
-- joined to all other tables, but instead follows a chain.
|
-- joined to all other tables, but instead follows a chain.
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM (
|
FROM (
|
||||||
SELECT users_table.user_id
|
SELECT users_table.user_id
|
||||||
FROM users_table
|
FROM users_table
|
||||||
|
@ -187,8 +197,8 @@ JOIN (
|
||||||
LOG: join order: [ "users_table" ][ local partition join "events_table" ][ local partition join "users_table" ][ local partition join "events_table" ]
|
LOG: join order: [ "users_table" ][ local partition join "events_table" ][ local partition join "users_table" ][ local partition join "events_table" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
|
|
|
@ -11,7 +11,8 @@ SET client_min_messages TO LOG;
|
||||||
-- except that more data has been loaded into customer and part tables. Therefore,
|
-- except that more data has been loaded into customer and part tables. Therefore,
|
||||||
-- we will apply different distributed join strategies for these queries.
|
-- we will apply different distributed join strategies for these queries.
|
||||||
-- Query #6 from the TPC-H decision support benchmark
|
-- Query #6 from the TPC-H decision support benchmark
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice * l_discount) as revenue
|
sum(l_extendedprice * l_discount) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem
|
lineitem
|
||||||
|
@ -23,13 +24,14 @@ WHERE
|
||||||
LOG: join order: [ "lineitem" ]
|
LOG: join order: [ "lineitem" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Query #3 from the TPC-H decision support benchmark
|
-- Query #3 from the TPC-H decision support benchmark
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
l_orderkey,
|
l_orderkey,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
o_orderdate,
|
o_orderdate,
|
||||||
|
@ -54,16 +56,17 @@ ORDER BY
|
||||||
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single range partition join "customer_append" ]
|
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single range partition join "customer_append" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort
|
||||||
Sort Key: (sum(remote_scan.revenue)) DESC, remote_scan.o_orderdate
|
Sort Key: (sum(remote_scan.revenue)) DESC, remote_scan.o_orderdate
|
||||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
-> HashAggregate
|
||||||
Group Key: remote_scan.l_orderkey, remote_scan.o_orderdate, remote_scan.o_shippriority
|
Group Key: remote_scan.l_orderkey, remote_scan.o_orderdate, remote_scan.o_shippriority
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(6 rows)
|
(6 rows)
|
||||||
|
|
||||||
-- Query #10 from the TPC-H decision support benchmark
|
-- Query #10 from the TPC-H decision support benchmark
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
c_custkey,
|
c_custkey,
|
||||||
c_name,
|
c_name,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
|
@ -97,16 +100,17 @@ ORDER BY
|
||||||
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single range partition join "customer_append" ][ reference join "nation" ]
|
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single range partition join "customer_append" ][ reference join "nation" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort
|
||||||
Sort Key: (sum(remote_scan.revenue)) DESC
|
Sort Key: (sum(remote_scan.revenue)) DESC
|
||||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
-> 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
|
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) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(6 rows)
|
(6 rows)
|
||||||
|
|
||||||
-- Query #19 from the TPC-H decision support benchmark (modified)
|
-- Query #19 from the TPC-H decision support benchmark (modified)
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice* (1 - l_discount)) as revenue
|
sum(l_extendedprice* (1 - l_discount)) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem,
|
lineitem,
|
||||||
|
@ -138,13 +142,14 @@ WHERE
|
||||||
LOG: join order: [ "lineitem" ][ single range partition join "part_append" ]
|
LOG: join order: [ "lineitem" ][ single range partition join "part_append" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Query to test multiple re-partition jobs in a single query
|
-- Query to test multiple re-partition jobs in a single query
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
l_partkey, count(*)
|
l_partkey, count(*)
|
||||||
FROM
|
FROM
|
||||||
lineitem, part_append, orders, customer_append
|
lineitem, part_append, orders, customer_append
|
||||||
|
@ -157,9 +162,9 @@ GROUP BY
|
||||||
LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single range partition join "part_append" ][ single range partition join "customer_append" ]
|
LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single range partition join "part_append" ][ single range partition join "customer_append" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
HashAggregate
|
||||||
Group Key: remote_scan.l_partkey
|
Group Key: remote_scan.l_partkey
|
||||||
-> Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Task-Tracker)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(4 rows)
|
(4 rows)
|
||||||
|
|
||||||
|
|
|
@ -6,7 +6,8 @@ SET citus.explain_distributed_queries TO off;
|
||||||
SET citus.log_multi_join_order TO TRUE;
|
SET citus.log_multi_join_order TO TRUE;
|
||||||
SET client_min_messages TO LOG;
|
SET client_min_messages TO LOG;
|
||||||
-- Query #6 from the TPC-H decision support benchmark
|
-- Query #6 from the TPC-H decision support benchmark
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice * l_discount) as revenue
|
sum(l_extendedprice * l_discount) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem
|
lineitem
|
||||||
|
@ -18,13 +19,14 @@ WHERE
|
||||||
LOG: join order: [ "lineitem" ]
|
LOG: join order: [ "lineitem" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Query #3 from the TPC-H decision support benchmark
|
-- Query #3 from the TPC-H decision support benchmark
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
l_orderkey,
|
l_orderkey,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
o_orderdate,
|
o_orderdate,
|
||||||
|
@ -49,16 +51,17 @@ ORDER BY
|
||||||
LOG: join order: [ "orders" ][ reference join "customer" ][ local partition join "lineitem" ]
|
LOG: join order: [ "orders" ][ reference join "customer" ][ local partition join "lineitem" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort
|
||||||
Sort Key: (sum(remote_scan.revenue)) DESC, remote_scan.o_orderdate
|
Sort Key: (sum(remote_scan.revenue)) DESC, remote_scan.o_orderdate
|
||||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
-> HashAggregate
|
||||||
Group Key: remote_scan.l_orderkey, remote_scan.o_orderdate, remote_scan.o_shippriority
|
Group Key: remote_scan.l_orderkey, remote_scan.o_orderdate, remote_scan.o_shippriority
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(6 rows)
|
(6 rows)
|
||||||
|
|
||||||
-- Query #10 from the TPC-H decision support benchmark
|
-- Query #10 from the TPC-H decision support benchmark
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
c_custkey,
|
c_custkey,
|
||||||
c_name,
|
c_name,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
|
@ -92,16 +95,17 @@ ORDER BY
|
||||||
LOG: join order: [ "orders" ][ reference join "customer" ][ reference join "nation" ][ local partition join "lineitem" ]
|
LOG: join order: [ "orders" ][ reference join "customer" ][ reference join "nation" ][ local partition join "lineitem" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort
|
||||||
Sort Key: (sum(remote_scan.revenue)) DESC
|
Sort Key: (sum(remote_scan.revenue)) DESC
|
||||||
-> HashAggregate (cost=0.00..0.00 rows=0 width=0)
|
-> 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
|
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 Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(6 rows)
|
(6 rows)
|
||||||
|
|
||||||
-- Query #19 from the TPC-H decision support benchmark (modified)
|
-- Query #19 from the TPC-H decision support benchmark (modified)
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice* (1 - l_discount)) as revenue
|
sum(l_extendedprice* (1 - l_discount)) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem,
|
lineitem,
|
||||||
|
@ -133,8 +137,8 @@ WHERE
|
||||||
LOG: join order: [ "lineitem" ][ reference join "part" ]
|
LOG: join order: [ "lineitem" ][ reference join "part" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
|
|
|
@ -68,7 +68,8 @@ DEBUG: Router planner does not support append-partitioned tables.
|
||||||
-- different type of columns including varchar, array types, composite types
|
-- different type of columns including varchar, array types, composite types
|
||||||
-- etc. This is in response to a bug we had where we were not able to resolve
|
-- etc. This is in response to a bug we had where we were not able to resolve
|
||||||
-- correct operator types for some kind of column types.
|
-- correct operator types for some kind of column types.
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM array_partitioned_table table1, array_partitioned_table table2
|
FROM array_partitioned_table table1, array_partitioned_table table2
|
||||||
WHERE table1.array_column = table2.array_column;
|
WHERE table1.array_column = table2.array_column;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
|
@ -76,12 +77,13 @@ DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1
|
||||||
DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}]
|
DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
||||||
WHERE table1.composite_column = table2.composite_column;
|
WHERE table1.composite_column = table2.composite_column;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
|
@ -89,13 +91,14 @@ DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)]
|
||||||
DEBUG: join prunable for intervals [(c,5,d),(d,6,e)] and [(a,3,b),(b,4,c)]
|
DEBUG: join prunable for intervals [(c,5,d),(d,6,e)] and [(a,3,b),(b,4,c)]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Test that large table joins on partition varchar columns work
|
-- Test that large table joins on partition varchar columns work
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM varchar_partitioned_table table1, varchar_partitioned_table table2
|
FROM varchar_partitioned_table table1, varchar_partitioned_table table2
|
||||||
WHERE table1.varchar_column = table2.varchar_column;
|
WHERE table1.varchar_column = table2.varchar_column;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
|
@ -103,8 +106,8 @@ DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U
|
||||||
DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6]
|
DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
|
|
|
@ -163,13 +163,14 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
|
|
||||||
-- Query that should result in a repartition join on UDT column.
|
-- Query that should result in a repartition join on UDT column.
|
||||||
SET citus.log_multi_join_order = true;
|
SET citus.log_multi_join_order = true;
|
||||||
EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||||
WHERE repartition_udt.pk > 1;
|
WHERE repartition_udt.pk > 1;
|
||||||
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
|
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
Custom Scan (Citus Task-Tracker)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: None, not supported for re-partition queries
|
Tasks Shown: None, not supported for re-partition queries
|
||||||
-> MapMergeJob
|
-> MapMergeJob
|
||||||
|
|
|
@ -28,7 +28,7 @@ LIMIT 1;
|
||||||
1 | 3.2857142857142857
|
1 | 3.2857142857142857
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
EXPLAIN
|
EXPLAIN (COSTS OFF)
|
||||||
SELECT user_id, avg(value_1)
|
SELECT user_id, avg(value_1)
|
||||||
FROM users_table
|
FROM users_table
|
||||||
GROUP BY user_id
|
GROUP BY user_id
|
||||||
|
@ -36,20 +36,20 @@ ORDER BY avg(value_1) DESC
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit (cost=0.00..0.00 rows=0 width=0)
|
Limit
|
||||||
-> Sort (cost=0.00..0.00 rows=0 width=0)
|
-> Sort
|
||||||
Sort Key: remote_scan.avg DESC
|
Sort Key: remote_scan.avg DESC
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: One of 4
|
Tasks Shown: One of 4
|
||||||
-> Task
|
-> Task
|
||||||
Node: host=localhost port=xxxxx dbname=regression
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
-> Limit (cost=1.53..1.53 rows=1 width=36)
|
-> Limit
|
||||||
-> Sort (cost=1.53..1.53 rows=2 width=36)
|
-> Sort
|
||||||
Sort Key: (avg(value_1)) DESC
|
Sort Key: (avg(value_1)) DESC
|
||||||
-> HashAggregate (cost=1.50..1.52 rows=2 width=36)
|
-> HashAggregate
|
||||||
Group Key: user_id
|
Group Key: user_id
|
||||||
-> Seq Scan on users_table_1400256 users_table (cost=0.00..1.33 rows=33 width=8)
|
-> Seq Scan on users_table_1400256 users_table
|
||||||
(14 rows)
|
(14 rows)
|
||||||
|
|
||||||
SELECT user_id, avg(value_1) + 1
|
SELECT user_id, avg(value_1) + 1
|
||||||
|
@ -96,23 +96,23 @@ ORDER BY 2 DESC;
|
||||||
1 | 10.2857142857142857
|
1 | 10.2857142857142857
|
||||||
(6 rows)
|
(6 rows)
|
||||||
|
|
||||||
EXPLAIN
|
EXPLAIN (COSTS OFF)
|
||||||
SELECT user_id, avg(value_1) + count(value_2)
|
SELECT user_id, avg(value_1) + count(value_2)
|
||||||
FROM users_table
|
FROM users_table
|
||||||
GROUP BY user_id
|
GROUP BY user_id
|
||||||
ORDER BY 2 DESC;
|
ORDER BY 2 DESC;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort (cost=0.00..0.00 rows=0 width=0)
|
Sort
|
||||||
Sort Key: remote_scan."?column?" DESC
|
Sort Key: remote_scan."?column?" DESC
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: One of 4
|
Tasks Shown: One of 4
|
||||||
-> Task
|
-> Task
|
||||||
Node: host=localhost port=xxxxx dbname=regression
|
Node: host=localhost port=xxxxx dbname=regression
|
||||||
-> HashAggregate (cost=1.58..1.61 rows=2 width=36)
|
-> HashAggregate
|
||||||
Group Key: user_id
|
Group Key: user_id
|
||||||
-> Seq Scan on users_table_1400256 users_table (cost=0.00..1.33 rows=33 width=12)
|
-> Seq Scan on users_table_1400256 users_table
|
||||||
(10 rows)
|
(10 rows)
|
||||||
|
|
||||||
SELECT user_id, avg(value_1) + count(value_2)
|
SELECT user_id, avg(value_1) + count(value_2)
|
||||||
|
|
|
@ -106,6 +106,7 @@ 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: assigned task to node localhost:xxxxx
|
||||||
DEBUG: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
|
DEBUG: master 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 3
|
DEBUG: completed cleanup query for job 3
|
||||||
DEBUG: completed cleanup query for job 2
|
DEBUG: completed cleanup query for job 2
|
||||||
|
@ -209,6 +210,7 @@ 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: assigned task to node localhost:xxxxx
|
||||||
DEBUG: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
|
DEBUG: master 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 6
|
DEBUG: completed cleanup query for job 6
|
||||||
DEBUG: completed cleanup query for job 4
|
DEBUG: completed cleanup query for job 4
|
||||||
|
@ -283,6 +285,7 @@ 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: assigned task to node localhost:xxxxx
|
||||||
DEBUG: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
|
DEBUG: master 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 9
|
DEBUG: completed cleanup query for job 9
|
||||||
DEBUG: completed cleanup query for job 7
|
DEBUG: completed cleanup query for job 7
|
||||||
|
@ -359,6 +362,7 @@ 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: assigned task to node localhost:xxxxx
|
||||||
DEBUG: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
|
DEBUG: master 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 12
|
DEBUG: completed cleanup query for job 12
|
||||||
DEBUG: completed cleanup query for job 10
|
DEBUG: completed cleanup query for job 10
|
||||||
|
@ -433,6 +437,7 @@ 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: assigned task to node localhost:xxxxx
|
||||||
DEBUG: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
|
DEBUG: master 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 15
|
DEBUG: completed cleanup query for job 15
|
||||||
DEBUG: completed cleanup query for job 13
|
DEBUG: completed cleanup query for job 13
|
||||||
|
@ -517,6 +522,7 @@ 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: assigned task to node localhost:xxxxx
|
||||||
DEBUG: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
|
DEBUG: master 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 18
|
DEBUG: completed cleanup query for job 18
|
||||||
DEBUG: completed cleanup query for job 16
|
DEBUG: completed cleanup query for job 16
|
||||||
|
|
|
@ -164,13 +164,14 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
-- Query that should result in a repartition join on UDT column.
|
-- Query that should result in a repartition join on UDT column.
|
||||||
SET citus.task_executor_type = 'task-tracker';
|
SET citus.task_executor_type = 'task-tracker';
|
||||||
SET citus.log_multi_join_order = true;
|
SET citus.log_multi_join_order = true;
|
||||||
EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||||
WHERE repartition_udt.pk > 1;
|
WHERE repartition_udt.pk > 1;
|
||||||
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
|
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Custom Scan (Citus Task-Tracker) (cost=0.00..0.00 rows=0 width=0)
|
Custom Scan (Citus Task-Tracker)
|
||||||
Task Count: 4
|
Task Count: 4
|
||||||
Tasks Shown: None, not supported for re-partition queries
|
Tasks Shown: None, not supported for re-partition queries
|
||||||
-> MapMergeJob
|
-> MapMergeJob
|
||||||
|
|
|
@ -236,9 +236,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 2 DESC, 1;
|
ORDER BY 2 DESC, 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
Sort Key: remote_scan.count DESC, remote_scan.l_orderkey
|
|
||||||
-> Unique
|
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.count DESC, remote_scan.l_orderkey
|
Sort Key: remote_scan.count DESC, remote_scan.l_orderkey
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -251,7 +249,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
Filter: (count(*) > 5)
|
Filter: (count(*) > 5)
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
Filter: (l_orderkey < 200)
|
Filter: (l_orderkey < 200)
|
||||||
(15 rows)
|
(13 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- distinct on aggregate of group by columns, we try to check whether we handle
|
-- distinct on aggregate of group by columns, we try to check whether we handle
|
||||||
|
@ -278,10 +276,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 1;
|
ORDER BY 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
|
-> Sort
|
||||||
Sort Key: (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
Sort Key: (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
||||||
-> HashAggregate
|
|
||||||
Group Key: COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint)
|
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.worker_column_2, remote_scan.worker_column_3
|
Group Key: remote_scan.worker_column_2, remote_scan.worker_column_3
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -292,7 +289,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(14 rows)
|
(13 rows)
|
||||||
|
|
||||||
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique
|
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique
|
||||||
-- instead of aggregate plan node to handle distinct.
|
-- instead of aggregate plan node to handle distinct.
|
||||||
|
@ -304,9 +301,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 1;
|
ORDER BY 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
Sort Key: (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
|
||||||
-> Unique
|
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
Sort Key: (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
||||||
-> GroupAggregate
|
-> GroupAggregate
|
||||||
|
@ -321,7 +316,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(17 rows)
|
(15 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- Now we have only part of group clause columns in distinct, yet it is still not
|
-- Now we have only part of group clause columns in distinct, yet it is still not
|
||||||
|
@ -355,10 +350,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.l_suppkey
|
Sort Key: remote_scan.l_suppkey, (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
||||||
-> HashAggregate
|
|
||||||
Group Key: remote_scan.l_suppkey, COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint)
|
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.l_suppkey, remote_scan.worker_column_3
|
Group Key: remote_scan.l_suppkey, remote_scan.worker_column_3
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -369,7 +363,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(15 rows)
|
(14 rows)
|
||||||
|
|
||||||
-- check the plan if the hash aggreate is disabled. Similar to the explain of
|
-- check the plan if the hash aggreate is disabled. Similar to the explain of
|
||||||
-- the query above.
|
-- the query above.
|
||||||
|
@ -383,8 +377,6 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
-> Sort
|
|
||||||
Sort Key: remote_scan.l_suppkey
|
|
||||||
-> Unique
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.l_suppkey, (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
Sort Key: remote_scan.l_suppkey, (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
||||||
|
@ -400,7 +392,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(18 rows)
|
(16 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- Similar to the above query, not with count but avg. Only difference with the
|
-- Similar to the above query, not with count but avg. Only difference with the
|
||||||
|
@ -435,10 +427,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))
|
Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))
|
||||||
-> HashAggregate
|
|
||||||
Group Key: remote_scan.l_suppkey, (pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1))
|
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.l_suppkey, remote_scan.worker_column_4
|
Group Key: remote_scan.l_suppkey, remote_scan.worker_column_4
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -449,7 +440,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(15 rows)
|
(14 rows)
|
||||||
|
|
||||||
-- check the plan if the hash aggreate is disabled. This explain errors out due
|
-- check the plan if the hash aggreate is disabled. This explain errors out due
|
||||||
-- to a bug right now, expectation must be corrected after fixing it.
|
-- to a bug right now, expectation must be corrected after fixing it.
|
||||||
|
@ -463,8 +454,6 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
-> Sort
|
|
||||||
Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))
|
|
||||||
-> Unique
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))
|
Sort Key: remote_scan.l_suppkey, ((pg_catalog.sum(remote_scan.avg) / pg_catalog.sum(remote_scan.avg_1)))
|
||||||
|
@ -480,7 +469,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(18 rows)
|
(16 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- Similar to the above query but with distinct on
|
-- Similar to the above query but with distinct on
|
||||||
|
@ -589,10 +578,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision))
|
Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision))
|
||||||
-> HashAggregate
|
|
||||||
Group Key: (sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision)
|
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4
|
Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -603,7 +591,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(15 rows)
|
(14 rows)
|
||||||
|
|
||||||
-- check the plan if the hash aggreate is disabled. This explain errors out due
|
-- check the plan if the hash aggreate is disabled. This explain errors out due
|
||||||
-- to a bug right now, expectation must be corrected after fixing it.
|
-- to a bug right now, expectation must be corrected after fixing it.
|
||||||
|
@ -617,8 +605,6 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
-> Sort
|
|
||||||
Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision))
|
|
||||||
-> Unique
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision))
|
Sort Key: ((sum(remote_scan.avg) / (pg_catalog.sum(remote_scan.avg_1))::double precision))
|
||||||
|
@ -634,7 +620,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(18 rows)
|
(16 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- expression among aggregations.
|
-- expression among aggregations.
|
||||||
|
@ -667,10 +653,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)))
|
Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)))
|
||||||
-> HashAggregate
|
|
||||||
Group Key: ((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint))
|
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4
|
Group Key: remote_scan.worker_column_3, remote_scan.worker_column_4
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -681,7 +666,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(15 rows)
|
(14 rows)
|
||||||
|
|
||||||
-- check the plan if the hash aggreate is disabled. This explain errors out due
|
-- check the plan if the hash aggreate is disabled. This explain errors out due
|
||||||
-- to a bug right now, expectation must be corrected after fixing it.
|
-- to a bug right now, expectation must be corrected after fixing it.
|
||||||
|
@ -695,8 +680,6 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
-> Sort
|
|
||||||
Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)))
|
|
||||||
-> Unique
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)))
|
Sort Key: (((pg_catalog.sum(remote_scan.dis))::bigint + COALESCE((pg_catalog.sum(remote_scan.dis_1))::bigint, '0'::bigint)))
|
||||||
|
@ -712,7 +695,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey, l_linenumber
|
Group Key: l_suppkey, l_linenumber
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(18 rows)
|
(16 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- distinct on all columns, note Group By columns guarantees uniqueness of the
|
-- distinct on all columns, note Group By columns guarantees uniqueness of the
|
||||||
|
@ -777,8 +760,6 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
-> Sort
|
|
||||||
Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey
|
|
||||||
-> Unique
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey, remote_scan.l_suppkey, remote_scan.l_linenumber, remote_scan.l_quantity, remote_scan.l_extendedprice, remote_scan.l_discount, remote_scan.l_tax, remote_scan.l_returnflag, remote_scan.l_linestatus, remote_scan.l_shipdate, remote_scan.l_commitdate, remote_scan.l_receiptdate, remote_scan.l_shipinstruct, remote_scan.l_shipmode, remote_scan.l_comment
|
Sort Key: remote_scan.l_orderkey, remote_scan.l_partkey, remote_scan.l_suppkey, remote_scan.l_linenumber, remote_scan.l_quantity, remote_scan.l_extendedprice, remote_scan.l_discount, remote_scan.l_tax, remote_scan.l_returnflag, remote_scan.l_linestatus, remote_scan.l_shipdate, remote_scan.l_commitdate, remote_scan.l_receiptdate, remote_scan.l_shipinstruct, remote_scan.l_shipmode, remote_scan.l_comment
|
||||||
|
@ -794,7 +775,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
Sort Key: l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(18 rows)
|
(16 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- distinct on count distinct
|
-- distinct on count distinct
|
||||||
|
@ -866,9 +847,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 1,2;
|
ORDER BY 1,2;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
Sort Key: remote_scan.count, remote_scan.count_1
|
|
||||||
-> Unique
|
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.count, remote_scan.count_1
|
Sort Key: remote_scan.count, remote_scan.count_1
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -881,7 +860,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: l_orderkey
|
Sort Key: l_orderkey
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(15 rows)
|
(13 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- distinct on aggregation with filter and expression
|
-- distinct on aggregation with filter and expression
|
||||||
|
@ -906,10 +885,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 1;
|
ORDER BY 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
|
-> Sort
|
||||||
Sort Key: (ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision))
|
Sort Key: (ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision))
|
||||||
-> HashAggregate
|
|
||||||
Group Key: ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision)
|
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.worker_column_2
|
Group Key: remote_scan.worker_column_2
|
||||||
-> Custom Scan (Citus Adaptive)
|
-> Custom Scan (Citus Adaptive)
|
||||||
|
@ -920,7 +898,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey
|
Group Key: l_suppkey
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(14 rows)
|
(13 rows)
|
||||||
|
|
||||||
-- check the plan if the hash aggreate is disabled
|
-- check the plan if the hash aggreate is disabled
|
||||||
SET enable_hashagg TO off;
|
SET enable_hashagg TO off;
|
||||||
|
@ -931,9 +909,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 1;
|
ORDER BY 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
Sort Key: (ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision))
|
|
||||||
-> Unique
|
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: (ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision))
|
Sort Key: (ceil(((COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint) / 2))::double precision))
|
||||||
-> GroupAggregate
|
-> GroupAggregate
|
||||||
|
@ -948,7 +924,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_suppkey
|
Group Key: l_suppkey
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(17 rows)
|
(15 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- explain the query to see actual plan with array_agg aggregation.
|
-- explain the query to see actual plan with array_agg aggregation.
|
||||||
|
@ -988,8 +964,6 @@ EXPLAIN (COSTS FALSE)
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Limit
|
Limit
|
||||||
-> Sort
|
|
||||||
Sort Key: remote_scan.array_length
|
|
||||||
-> Unique
|
-> Unique
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: remote_scan.array_length, remote_scan.array_agg
|
Sort Key: remote_scan.array_length, remote_scan.array_agg
|
||||||
|
@ -1003,7 +977,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> Sort
|
-> Sort
|
||||||
Sort Key: l_orderkey
|
Sort Key: l_orderkey
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(16 rows)
|
(14 rows)
|
||||||
|
|
||||||
SET enable_hashagg TO on;
|
SET enable_hashagg TO on;
|
||||||
-- distinct on non-partition column with aggregate
|
-- distinct on non-partition column with aggregate
|
||||||
|
@ -1037,8 +1011,9 @@ EXPLAIN (COSTS FALSE)
|
||||||
ORDER BY 1;
|
ORDER BY 1;
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Sort
|
Unique
|
||||||
Sort Key: remote_scan.l_partkey
|
-> Sort
|
||||||
|
Sort Key: remote_scan.l_partkey, (COALESCE((pg_catalog.sum(remote_scan.count))::bigint, '0'::bigint))
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: remote_scan.l_partkey
|
Group Key: remote_scan.l_partkey
|
||||||
Filter: (COALESCE((pg_catalog.sum(remote_scan.worker_column_3))::bigint, '0'::bigint) > 2)
|
Filter: (COALESCE((pg_catalog.sum(remote_scan.worker_column_3))::bigint, '0'::bigint) > 2)
|
||||||
|
@ -1050,7 +1025,7 @@ EXPLAIN (COSTS FALSE)
|
||||||
-> HashAggregate
|
-> HashAggregate
|
||||||
Group Key: l_partkey
|
Group Key: l_partkey
|
||||||
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
|
||||||
(13 rows)
|
(14 rows)
|
||||||
|
|
||||||
-- distinct on non-partition column and avg
|
-- distinct on non-partition column and avg
|
||||||
SELECT DISTINCT l_partkey, avg(l_linenumber)
|
SELECT DISTINCT l_partkey, avg(l_linenumber)
|
||||||
|
|
|
@ -68,53 +68,53 @@ BEGIN;
|
||||||
SET client_min_messages TO DEBUG3;
|
SET client_min_messages TO DEBUG3;
|
||||||
-- First test the default greedy task assignment policy
|
-- First test the default greedy task assignment policy
|
||||||
SET citus.task_assignment_policy TO 'greedy';
|
SET citus.task_assignment_policy TO 'greedy';
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
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: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
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: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
-- Next test the first-replica task assignment policy
|
-- Next test the first-replica task assignment policy
|
||||||
SET citus.task_assignment_policy TO 'first-replica';
|
SET citus.task_assignment_policy TO 'first-replica';
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
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: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
DEBUG: Router planner does not support append-partitioned tables.
|
DEBUG: Router planner does not support append-partitioned tables.
|
||||||
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: assigned task to node localhost:xxxxx
|
DEBUG: assigned task to node localhost:xxxxx
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
Aggregate (cost=0.00..0.00 rows=0 width=0)
|
Aggregate
|
||||||
-> Custom Scan (Citus Adaptive) (cost=0.00..0.00 rows=0 width=0)
|
-> Custom Scan (Citus Adaptive)
|
||||||
explain statements for distributed queries are not enabled
|
explain statements for distributed queries are not enabled
|
||||||
(3 rows)
|
(3 rows)
|
||||||
|
|
||||||
|
|
|
@ -401,7 +401,8 @@ BEGIN;
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
-- coordinator INSERT .. SELECT
|
-- coordinator INSERT .. SELECT
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO table_2 SELECT * FROM table_1 OFFSET 0;
|
-- We use offset 1 to make sure the result needs to be pulled to the coordinator, offset 0 would be optimized away
|
||||||
|
INSERT INTO table_2 SELECT * FROM table_1 OFFSET 1;
|
||||||
SELECT * FROM relation_acesses WHERE table_name IN ('table_1', 'table_2') ORDER BY 1;
|
SELECT * FROM relation_acesses WHERE table_name IN ('table_1', 'table_2') ORDER BY 1;
|
||||||
table_name | select_access | dml_access | ddl_access
|
table_name | select_access | dml_access | ddl_access
|
||||||
---------------------------------------------------------------------
|
---------------------------------------------------------------------
|
||||||
|
|
|
@ -84,7 +84,7 @@ test: multi_subquery_union multi_subquery_in_where_clause multi_subquery_misc
|
||||||
test: multi_agg_distinct multi_agg_approximate_distinct multi_limit_clause_approximate multi_outer_join_reference multi_single_relation_subquery multi_prepare_plsql
|
test: multi_agg_distinct multi_agg_approximate_distinct multi_limit_clause_approximate multi_outer_join_reference multi_single_relation_subquery multi_prepare_plsql
|
||||||
test: multi_reference_table multi_select_for_update relation_access_tracking
|
test: multi_reference_table multi_select_for_update relation_access_tracking
|
||||||
test: custom_aggregate_support aggregate_support
|
test: custom_aggregate_support aggregate_support
|
||||||
test: multi_average_expression multi_working_columns multi_having_pushdown
|
test: multi_average_expression multi_working_columns multi_having_pushdown having_subquery
|
||||||
test: multi_array_agg multi_limit_clause multi_orderby_limit_pushdown
|
test: multi_array_agg multi_limit_clause multi_orderby_limit_pushdown
|
||||||
test: multi_jsonb_agg multi_jsonb_object_agg multi_json_agg multi_json_object_agg bool_agg ch_bench_having ch_bench_subquery_repartition chbenchmark_all_queries expression_reference_join
|
test: multi_jsonb_agg multi_jsonb_object_agg multi_json_agg multi_json_object_agg bool_agg ch_bench_having ch_bench_subquery_repartition chbenchmark_all_queries expression_reference_join
|
||||||
test: multi_agg_type_conversion multi_count_type_conversion
|
test: multi_agg_type_conversion multi_count_type_conversion
|
||||||
|
|
|
@ -42,7 +42,8 @@ SELECT create_reference_table('ref_table');
|
||||||
|
|
||||||
|
|
||||||
-- single hash repartition after bcast joins
|
-- single hash repartition after bcast joins
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
count(*)
|
count(*)
|
||||||
FROM
|
FROM
|
||||||
ref_table r1, single_hash_repartition_second t1, single_hash_repartition_first t2
|
ref_table r1, single_hash_repartition_second t1, single_hash_repartition_first t2
|
||||||
|
@ -50,7 +51,8 @@ WHERE
|
||||||
r1.id = t1.id AND t2.sum = t1.id;
|
r1.id = t1.id AND t2.sum = t1.id;
|
||||||
|
|
||||||
-- a more complicated join order, first colocated join, later single hash repartition join
|
-- a more complicated join order, first colocated join, later single hash repartition join
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
count(*)
|
count(*)
|
||||||
FROM
|
FROM
|
||||||
single_hash_repartition_first t1, single_hash_repartition_first t2, single_hash_repartition_second t3
|
single_hash_repartition_first t1, single_hash_repartition_first t2, single_hash_repartition_second t3
|
||||||
|
|
|
@ -32,13 +32,15 @@ group by s_i_id
|
||||||
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock);
|
having sum(s_order_cnt) > (select max(s_order_cnt) - 3 as having_query from stock);
|
||||||
|
|
||||||
|
|
||||||
explain select s_i_id, sum(s_order_cnt) as ordercount
|
explain (costs false)
|
||||||
|
select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
from stock s
|
from stock s
|
||||||
group by s_i_id
|
group by s_i_id
|
||||||
having (select true)
|
having (select true)
|
||||||
order by s_i_id;
|
order by s_i_id;
|
||||||
|
|
||||||
explain select s_i_id, sum(s_order_cnt) as ordercount
|
explain (costs false)
|
||||||
|
select s_i_id, sum(s_order_cnt) as ordercount
|
||||||
from stock s
|
from stock s
|
||||||
group by s_i_id
|
group by s_i_id
|
||||||
having (select true);
|
having (select true);
|
||||||
|
|
|
@ -473,7 +473,7 @@ GROUP BY
|
||||||
key
|
key
|
||||||
HAVING
|
HAVING
|
||||||
(count(*) > (SELECT max FROM cte_1))
|
(count(*) > (SELECT max FROM cte_1))
|
||||||
ORDER BY 2 DESC
|
ORDER BY 2 DESC, 1 DESC
|
||||||
LIMIT 5;
|
LIMIT 5;
|
||||||
|
|
||||||
-- cte used in ORDER BY just works fine
|
-- cte used in ORDER BY just works fine
|
||||||
|
|
|
@ -0,0 +1,27 @@
|
||||||
|
-- Testing a having clause that could have been a where clause between a distributed table
|
||||||
|
-- and a reference table. This query was the cause for intermediate results not being
|
||||||
|
-- available during the replace of the planner for the master query with the standard
|
||||||
|
-- planner.
|
||||||
|
-- Since the having clause could have been a where clause the having clause on the grouping
|
||||||
|
-- on the coordinator is replaced with a Result node containing a One-time filter if the
|
||||||
|
-- having qual (one-time filter works because the query doesn't change with the tuples
|
||||||
|
-- returned from below).
|
||||||
|
SELECT count(*),
|
||||||
|
o_orderstatus
|
||||||
|
FROM orders
|
||||||
|
GROUP BY 2
|
||||||
|
HAVING (
|
||||||
|
SELECT count(*)
|
||||||
|
FROM customer
|
||||||
|
) > 0;
|
||||||
|
|
||||||
|
-- lets pin the plan in the test as well
|
||||||
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*),
|
||||||
|
o_orderstatus
|
||||||
|
FROM orders
|
||||||
|
GROUP BY 2
|
||||||
|
HAVING (
|
||||||
|
SELECT count(*)
|
||||||
|
FROM customer
|
||||||
|
) > 0;
|
|
@ -442,8 +442,9 @@ ROLLBACK;
|
||||||
-- test with INSERT SELECT via coordinator
|
-- test with INSERT SELECT via coordinator
|
||||||
|
|
||||||
-- INSERT .. SELECT via coordinator that doesn't have any intermediate results
|
-- INSERT .. SELECT via coordinator that doesn't have any intermediate results
|
||||||
|
-- We use offset 1 to make sure the result needs to be pulled to the coordinator, offset 0 would be optimized away
|
||||||
INSERT INTO table_1
|
INSERT INTO table_1
|
||||||
SELECT * FROM table_2 OFFSET 0;
|
SELECT * FROM table_2 OFFSET 1;
|
||||||
|
|
||||||
-- INSERT .. SELECT via coordinator which has intermediate result,
|
-- INSERT .. SELECT via coordinator which has intermediate result,
|
||||||
-- and can be pruned to a single worker because the final query is on
|
-- and can be pruned to a single worker because the final query is on
|
||||||
|
|
|
@ -66,7 +66,8 @@ SELECT create_distributed_table('customer_hash', 'c_custkey');
|
||||||
SET client_min_messages TO DEBUG2;
|
SET client_min_messages TO DEBUG2;
|
||||||
-- The following query checks that we can correctly handle self-joins
|
-- The following query checks that we can correctly handle self-joins
|
||||||
|
|
||||||
EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT l1.l_quantity FROM lineitem l1, lineitem l2
|
||||||
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
|
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
|
||||||
|
|
||||||
SET client_min_messages TO LOG;
|
SET client_min_messages TO LOG;
|
||||||
|
@ -75,43 +76,52 @@ SET client_min_messages TO LOG;
|
||||||
-- particular, these queries check that we factorize out OR clauses if possible,
|
-- particular, these queries check that we factorize out OR clauses if possible,
|
||||||
-- and that we default to a cartesian product otherwise.
|
-- and that we default to a cartesian product otherwise.
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM lineitem, orders
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM lineitem, orders
|
||||||
WHERE (l_orderkey = o_orderkey AND l_quantity > 5)
|
WHERE (l_orderkey = o_orderkey AND l_quantity > 5)
|
||||||
OR (l_orderkey = o_orderkey AND l_quantity < 10);
|
OR (l_orderkey = o_orderkey AND l_quantity < 10);
|
||||||
|
|
||||||
EXPLAIN SELECT l_quantity FROM lineitem, orders
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT l_quantity FROM lineitem, orders
|
||||||
WHERE (l_orderkey = o_orderkey OR l_quantity > 5);
|
WHERE (l_orderkey = o_orderkey OR l_quantity > 5);
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM orders, lineitem_hash
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders, lineitem_hash
|
||||||
WHERE o_orderkey = l_orderkey;
|
WHERE o_orderkey = l_orderkey;
|
||||||
|
|
||||||
-- Verify we handle local joins between two hash-partitioned tables.
|
-- Verify we handle local joins between two hash-partitioned tables.
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash, lineitem_hash
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders_hash, lineitem_hash
|
||||||
WHERE o_orderkey = l_orderkey;
|
WHERE o_orderkey = l_orderkey;
|
||||||
|
|
||||||
-- Validate that we can handle broadcast joins with hash-partitioned tables.
|
-- Validate that we can handle broadcast joins with hash-partitioned tables.
|
||||||
EXPLAIN SELECT count(*) FROM customer_hash, nation
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM customer_hash, nation
|
||||||
WHERE c_nationkey = n_nationkey;
|
WHERE c_nationkey = n_nationkey;
|
||||||
|
|
||||||
-- Validate that we don't use a single-partition join method for a hash
|
-- Validate that we don't use a single-partition join method for a hash
|
||||||
-- re-partitioned table, thus preventing a partition of just the customer table.
|
-- re-partitioned table, thus preventing a partition of just the customer table.
|
||||||
EXPLAIN SELECT count(*) FROM orders, lineitem, customer_append
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders, lineitem, customer_append
|
||||||
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
|
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
|
||||||
|
|
||||||
-- Validate that we don't chose a single-partition join method with a
|
-- Validate that we don't chose a single-partition join method with a
|
||||||
-- hash-partitioned base table
|
-- hash-partitioned base table
|
||||||
EXPLAIN SELECT count(*) FROM orders, customer_hash
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders, customer_hash
|
||||||
WHERE c_custkey = o_custkey;
|
WHERE c_custkey = o_custkey;
|
||||||
|
|
||||||
-- Validate that we can re-partition a hash partitioned table to join with a
|
-- Validate that we can re-partition a hash partitioned table to join with a
|
||||||
-- range partitioned one.
|
-- range partitioned one.
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash, customer_append
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*) FROM orders_hash, customer_append
|
||||||
WHERE c_custkey = o_custkey;
|
WHERE c_custkey = o_custkey;
|
||||||
|
|
||||||
-- Validate a 4 way join that could be done locally is planned as such by the logical
|
-- Validate a 4 way join that could be done locally is planned as such by the logical
|
||||||
-- planner. It used to be planned as a repartition join due to no 1 table being directly
|
-- planner. It used to be planned as a repartition join due to no 1 table being directly
|
||||||
-- joined to all other tables, but instead follows a chain.
|
-- joined to all other tables, but instead follows a chain.
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM (
|
FROM (
|
||||||
SELECT users_table.user_id
|
SELECT users_table.user_id
|
||||||
FROM users_table
|
FROM users_table
|
||||||
|
|
|
@ -19,7 +19,8 @@ SET client_min_messages TO LOG;
|
||||||
|
|
||||||
-- Query #6 from the TPC-H decision support benchmark
|
-- Query #6 from the TPC-H decision support benchmark
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice * l_discount) as revenue
|
sum(l_extendedprice * l_discount) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem
|
lineitem
|
||||||
|
@ -31,7 +32,8 @@ WHERE
|
||||||
|
|
||||||
-- Query #3 from the TPC-H decision support benchmark
|
-- Query #3 from the TPC-H decision support benchmark
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
l_orderkey,
|
l_orderkey,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
o_orderdate,
|
o_orderdate,
|
||||||
|
@ -56,7 +58,8 @@ ORDER BY
|
||||||
|
|
||||||
-- Query #10 from the TPC-H decision support benchmark
|
-- Query #10 from the TPC-H decision support benchmark
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
c_custkey,
|
c_custkey,
|
||||||
c_name,
|
c_name,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
|
@ -90,7 +93,8 @@ ORDER BY
|
||||||
|
|
||||||
-- Query #19 from the TPC-H decision support benchmark (modified)
|
-- Query #19 from the TPC-H decision support benchmark (modified)
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice* (1 - l_discount)) as revenue
|
sum(l_extendedprice* (1 - l_discount)) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem,
|
lineitem,
|
||||||
|
@ -122,7 +126,8 @@ WHERE
|
||||||
|
|
||||||
-- Query to test multiple re-partition jobs in a single query
|
-- Query to test multiple re-partition jobs in a single query
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
l_partkey, count(*)
|
l_partkey, count(*)
|
||||||
FROM
|
FROM
|
||||||
lineitem, part_append, orders, customer_append
|
lineitem, part_append, orders, customer_append
|
||||||
|
|
|
@ -10,7 +10,8 @@ SET client_min_messages TO LOG;
|
||||||
|
|
||||||
-- Query #6 from the TPC-H decision support benchmark
|
-- Query #6 from the TPC-H decision support benchmark
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice * l_discount) as revenue
|
sum(l_extendedprice * l_discount) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem
|
lineitem
|
||||||
|
@ -22,7 +23,8 @@ WHERE
|
||||||
|
|
||||||
-- Query #3 from the TPC-H decision support benchmark
|
-- Query #3 from the TPC-H decision support benchmark
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
l_orderkey,
|
l_orderkey,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
o_orderdate,
|
o_orderdate,
|
||||||
|
@ -47,7 +49,8 @@ ORDER BY
|
||||||
|
|
||||||
-- Query #10 from the TPC-H decision support benchmark
|
-- Query #10 from the TPC-H decision support benchmark
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
c_custkey,
|
c_custkey,
|
||||||
c_name,
|
c_name,
|
||||||
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
sum(l_extendedprice * (1 - l_discount)) as revenue,
|
||||||
|
@ -81,7 +84,8 @@ ORDER BY
|
||||||
|
|
||||||
-- Query #19 from the TPC-H decision support benchmark (modified)
|
-- Query #19 from the TPC-H decision support benchmark (modified)
|
||||||
|
|
||||||
EXPLAIN SELECT
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT
|
||||||
sum(l_extendedprice* (1 - l_discount)) as revenue
|
sum(l_extendedprice* (1 - l_discount)) as revenue
|
||||||
FROM
|
FROM
|
||||||
lineitem,
|
lineitem,
|
||||||
|
|
|
@ -43,16 +43,19 @@ SELECT sum(l_linenumber), avg(l_linenumber)
|
||||||
-- etc. This is in response to a bug we had where we were not able to resolve
|
-- etc. This is in response to a bug we had where we were not able to resolve
|
||||||
-- correct operator types for some kind of column types.
|
-- correct operator types for some kind of column types.
|
||||||
|
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM array_partitioned_table table1, array_partitioned_table table2
|
FROM array_partitioned_table table1, array_partitioned_table table2
|
||||||
WHERE table1.array_column = table2.array_column;
|
WHERE table1.array_column = table2.array_column;
|
||||||
|
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
||||||
WHERE table1.composite_column = table2.composite_column;
|
WHERE table1.composite_column = table2.composite_column;
|
||||||
|
|
||||||
-- Test that large table joins on partition varchar columns work
|
-- Test that large table joins on partition varchar columns work
|
||||||
|
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT count(*)
|
||||||
FROM varchar_partitioned_table table1, varchar_partitioned_table table2
|
FROM varchar_partitioned_table table1, varchar_partitioned_table table2
|
||||||
WHERE table1.varchar_column = table2.varchar_column;
|
WHERE table1.varchar_column = table2.varchar_column;
|
||||||
|
|
|
@ -192,7 +192,8 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
-- Query that should result in a repartition join on UDT column.
|
-- Query that should result in a repartition join on UDT column.
|
||||||
SET citus.log_multi_join_order = true;
|
SET citus.log_multi_join_order = true;
|
||||||
|
|
||||||
EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||||
WHERE repartition_udt.pk > 1;
|
WHERE repartition_udt.pk > 1;
|
||||||
|
|
||||||
|
|
|
@ -18,7 +18,7 @@ GROUP BY user_id
|
||||||
ORDER BY avg(value_1) DESC
|
ORDER BY avg(value_1) DESC
|
||||||
LIMIT 1;
|
LIMIT 1;
|
||||||
|
|
||||||
EXPLAIN
|
EXPLAIN (COSTS OFF)
|
||||||
SELECT user_id, avg(value_1)
|
SELECT user_id, avg(value_1)
|
||||||
FROM users_table
|
FROM users_table
|
||||||
GROUP BY user_id
|
GROUP BY user_id
|
||||||
|
@ -48,7 +48,7 @@ FROM users_table
|
||||||
GROUP BY user_id
|
GROUP BY user_id
|
||||||
ORDER BY 2 DESC;
|
ORDER BY 2 DESC;
|
||||||
|
|
||||||
EXPLAIN
|
EXPLAIN (COSTS OFF)
|
||||||
SELECT user_id, avg(value_1) + count(value_2)
|
SELECT user_id, avg(value_1) + count(value_2)
|
||||||
FROM users_table
|
FROM users_table
|
||||||
GROUP BY user_id
|
GROUP BY user_id
|
||||||
|
|
|
@ -191,7 +191,8 @@ SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
SET citus.task_executor_type = 'task-tracker';
|
SET citus.task_executor_type = 'task-tracker';
|
||||||
SET citus.log_multi_join_order = true;
|
SET citus.log_multi_join_order = true;
|
||||||
|
|
||||||
EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
|
EXPLAIN (COSTS OFF)
|
||||||
|
SELECT * FROM repartition_udt JOIN repartition_udt_other
|
||||||
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
ON repartition_udt.udtcol = repartition_udt_other.udtcol
|
||||||
WHERE repartition_udt.pk > 1;
|
WHERE repartition_udt.pk > 1;
|
||||||
|
|
||||||
|
|
|
@ -85,17 +85,17 @@ SET client_min_messages TO DEBUG3;
|
||||||
|
|
||||||
SET citus.task_assignment_policy TO 'greedy';
|
SET citus.task_assignment_policy TO 'greedy';
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
|
|
||||||
-- Next test the first-replica task assignment policy
|
-- Next test the first-replica task assignment policy
|
||||||
|
|
||||||
SET citus.task_assignment_policy TO 'first-replica';
|
SET citus.task_assignment_policy TO 'first-replica';
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM task_assignment_test_table;
|
EXPLAIN (COSTS OFF) SELECT count(*) FROM task_assignment_test_table;
|
||||||
|
|
||||||
COMMIT;
|
COMMIT;
|
||||||
|
|
||||||
|
|
|
@ -234,7 +234,8 @@ ROLLBACK;
|
||||||
|
|
||||||
-- coordinator INSERT .. SELECT
|
-- coordinator INSERT .. SELECT
|
||||||
BEGIN;
|
BEGIN;
|
||||||
INSERT INTO table_2 SELECT * FROM table_1 OFFSET 0;
|
-- We use offset 1 to make sure the result needs to be pulled to the coordinator, offset 0 would be optimized away
|
||||||
|
INSERT INTO table_2 SELECT * FROM table_1 OFFSET 1;
|
||||||
SELECT * FROM relation_acesses WHERE table_name IN ('table_1', 'table_2') ORDER BY 1;
|
SELECT * FROM relation_acesses WHERE table_name IN ('table_1', 'table_2') ORDER BY 1;
|
||||||
ROLLBACK;
|
ROLLBACK;
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue