Merge pull request #1980 from citusdata/remove_boradcast_pr3_v2

Convert broadcast join to reference join
pull/2097/head
Burak Velioglu 2018-04-13 16:44:55 +03:00 committed by GitHub
commit f54ed5d3b0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 594 additions and 702 deletions

View File

@ -40,7 +40,6 @@ bool LogMultiJoinOrder = false; /* print join order as a debugging aid */
/* Function pointer type definition for join rule evaluation functions */
typedef JoinOrderNode *(*RuleEvalFunction) (JoinOrderNode *currentJoinNode,
TableEntry *candidateTable,
List *candidateShardList,
List *applicableJoinClauses,
JoinType joinType);
@ -53,10 +52,6 @@ static JoinOrderNode * CreateFirstJoinOrderNode(FromExpr *fromExpr,
List *tableEntryList);
static bool JoinExprListWalker(Node *node, List **joinList);
static bool ExtractLeftMostRangeTableIndex(Node *node, int *rangeTableIndex);
static List * MergeShardIntervals(List *leftShardIntervalList,
List *rightShardIntervalList, JoinType joinType);
static bool ShardIntervalsMatch(List *leftShardIntervalList,
List *rightShardIntervalList);
static List * JoinOrderForTable(TableEntry *firstTable, List *tableEntryList,
List *joinClauseList);
static List * BestJoinOrder(List *candidateJoinOrders);
@ -72,33 +67,26 @@ static TableEntry * FindTableEntry(List *tableEntryList, uint32 tableId);
static JoinOrderNode * EvaluateJoinRules(List *joinedTableList,
JoinOrderNode *currentJoinNode,
TableEntry *candidateTable,
List *candidateShardList,
List *joinClauseList, JoinType joinType);
static List * RangeTableIdList(List *tableList);
static RuleEvalFunction JoinRuleEvalFunction(JoinRuleType ruleType);
static char * JoinRuleName(JoinRuleType ruleType);
static JoinOrderNode * BroadcastJoin(JoinOrderNode *joinNode, TableEntry *candidateTable,
List *candidateShardList,
List *applicableJoinClauses,
JoinType joinType);
static JoinOrderNode * ReferenceJoin(JoinOrderNode *joinNode, TableEntry *candidateTable,
List *applicableJoinClauses, JoinType joinType);
static JoinOrderNode * LocalJoin(JoinOrderNode *joinNode, TableEntry *candidateTable,
List *candidateShardList, List *applicableJoinClauses,
JoinType joinType);
List *applicableJoinClauses, JoinType joinType);
static bool JoinOnColumns(Var *currentPartitioncolumn, Var *candidatePartitionColumn,
List *joinClauseList);
static JoinOrderNode * SinglePartitionJoin(JoinOrderNode *joinNode,
TableEntry *candidateTable,
List *candidateShardList,
List *applicableJoinClauses,
JoinType joinType);
static JoinOrderNode * DualPartitionJoin(JoinOrderNode *joinNode,
TableEntry *candidateTable,
List *candidateShardList,
List *applicableJoinClauses,
JoinType joinType);
static JoinOrderNode * CartesianProduct(JoinOrderNode *joinNode,
TableEntry *candidateTable,
List *candidateShardList,
List *applicableJoinClauses,
JoinType joinType);
static JoinOrderNode * MakeJoinOrderNode(TableEntry *tableEntry, JoinRuleType
@ -165,7 +153,6 @@ FixedJoinOrderList(FromExpr *fromExpr, List *tableEntryList)
RangeTblRef *nextRangeTableRef = NULL;
TableEntry *nextTable = NULL;
JoinOrderNode *nextJoinNode = NULL;
List *candidateShardList = NIL;
Node *rightArg = joinExpr->rarg;
/* get the table on the right hand side of the join */
@ -188,60 +175,11 @@ FixedJoinOrderList(FromExpr *fromExpr, List *tableEntryList)
joinClauseList = list_concat(joinClauseList, joinWhereClauseList);
}
/* get the sorted list of shards to check broadcast/local join possibility */
candidateShardList = LoadShardIntervalList(nextTable->relationId);
/* find the best join rule type */
nextJoinNode = EvaluateJoinRules(joinedTableList, currentJoinNode,
nextTable, candidateShardList,
joinClauseList, joinType);
nextTable, joinClauseList, joinType);
if (nextJoinNode->joinRuleType == BROADCAST_JOIN)
{
if (joinType == JOIN_RIGHT || joinType == JOIN_FULL)
{
/* the overall interval list is now the same as the right side */
nextJoinNode->shardIntervalList = candidateShardList;
}
else if (list_length(candidateShardList) == 1)
{
/* the overall interval list is now the same as the left side */
nextJoinNode->shardIntervalList = currentJoinNode->shardIntervalList;
}
else
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning on this "
"query"),
errdetail("Cannot perform outer joins with broadcast "
"joins of more than 1 shard"),
errhint("Set citus.large_table_shard_count to 1")));
}
}
else if (nextJoinNode->joinRuleType == LOCAL_PARTITION_JOIN)
{
/* shard interval lists must have 1-1 matching for local joins */
bool shardIntervalsMatch =
ShardIntervalsMatch(currentJoinNode->shardIntervalList,
candidateShardList);
if (shardIntervalsMatch)
{
nextJoinNode->shardIntervalList =
MergeShardIntervals(currentJoinNode->shardIntervalList,
candidateShardList,
joinType);
}
else
{
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot perform distributed planning on this "
"query"),
errdetail("Shards of relations in outer join queries "
"must have 1-to-1 shard partitioning")));
}
}
else
if (nextJoinNode->joinRuleType >= SINGLE_PARTITION_JOIN)
{
/* re-partitioning for OUTER joins is not implemented */
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
@ -299,8 +237,6 @@ CreateFirstJoinOrderNode(FromExpr *fromExpr, List *tableEntryList)
firstPartitionMethod,
firstTable);
firstJoinNode->shardIntervalList = LoadShardIntervalList(firstTable->relationId);
return firstJoinNode;
}
@ -413,85 +349,6 @@ ExtractLeftMostRangeTableIndex(Node *node, int *rangeTableIndex)
}
/*
* MergeShardIntervals merges given shard interval lists. It assumes that both lists
* have the same number of shard intervals, and each shard interval overlaps only with
* a corresponding shard interval from the other shard interval list. It uses union or
* intersection logic when merging two shard intervals depending on joinType.
*/
static List *
MergeShardIntervals(List *leftShardIntervalList, List *rightShardIntervalList,
JoinType joinType)
{
FmgrInfo *comparisonFunction = NULL;
ShardInterval *firstShardInterval = NULL;
Oid typeId = InvalidOid;
bool typeByValue = false;
int typeLen = 0;
ListCell *leftShardIntervalCell = NULL;
ListCell *rightShardIntervalCell = NULL;
List *mergedShardIntervalList = NIL;
bool shardUnion = IS_OUTER_JOIN(joinType);
Assert(list_length(leftShardIntervalList) > 0);
Assert(list_length(leftShardIntervalList) == list_length(rightShardIntervalList));
firstShardInterval = (ShardInterval *) linitial(leftShardIntervalList);
typeId = firstShardInterval->valueTypeId;
typeByValue = firstShardInterval->valueByVal;
typeLen = firstShardInterval->valueTypeLen;
comparisonFunction = GetFunctionInfo(typeId, BTREE_AM_OID, BTORDER_PROC);
forboth(leftShardIntervalCell, leftShardIntervalList,
rightShardIntervalCell, rightShardIntervalList)
{
ShardInterval *currentInterval = (ShardInterval *) lfirst(leftShardIntervalCell);
ShardInterval *nextInterval = (ShardInterval *) lfirst(rightShardIntervalCell);
ShardInterval *newShardInterval = NULL;
Datum currentMin = currentInterval->minValue;
Datum currentMax = currentInterval->maxValue;
newShardInterval = (ShardInterval *) palloc0(sizeof(ShardInterval));
CopyShardInterval(currentInterval, newShardInterval);
if (nextInterval->minValueExists)
{
Datum nextMin = nextInterval->minValue;
Datum comparisonDatum = CompareCall2(comparisonFunction, currentMin, nextMin);
int comparisonResult = DatumGetInt32(comparisonDatum);
bool nextMinSmaller = comparisonResult > 0;
bool nextMinLarger = comparisonResult < 0;
if ((shardUnion && nextMinSmaller) ||
(!shardUnion && nextMinLarger))
{
newShardInterval->minValue = datumCopy(nextMin, typeByValue, typeLen);
}
}
if (nextInterval->maxValueExists)
{
Datum nextMax = nextInterval->maxValue;
Datum comparisonDatum = CompareCall2(comparisonFunction, currentMax, nextMax);
int comparisonResult = DatumGetInt32(comparisonDatum);
bool nextMaxLarger = comparisonResult < 0;
bool nextMaxSmaller = comparisonResult > 0;
if ((shardUnion && nextMaxLarger) ||
(!shardUnion && nextMaxSmaller))
{
newShardInterval->maxValue = datumCopy(nextMax, typeByValue, typeLen);
}
}
mergedShardIntervalList = lappend(mergedShardIntervalList, newShardInterval);
}
return mergedShardIntervalList;
}
/*
* JoinOnColumns determines whether two columns are joined by a given join clause
* list.
@ -527,88 +384,6 @@ JoinOnColumns(Var *currentColumn, Var *candidateColumn, List *joinClauseList)
}
/*
* ShardIntervalsMatch returns true if provided shard interval has one-to-one
* matching. Shards intervals must be not empty, and their intervals musht be in
* ascending order of range min values. Shard interval ranges said to be matched
* only if (1) they have same number of shards, (2) a shard interval on the left
* side overlaps with corresponding shard on the right side, (3) a shard interval
* on the right side does not overlap with any other shard. The function does not
* compare a left shard with every right shard. It compares the left shard with the
* previous and next shards of the corresponding shard to check they to not overlap
* for optimization purposes.
*/
static bool
ShardIntervalsMatch(List *leftShardIntervalList, List *rightShardIntervalList)
{
int leftShardIntervalCount = list_length(leftShardIntervalList);
int rightShardIntervalCount = list_length(rightShardIntervalList);
ListCell *leftShardIntervalCell = NULL;
ListCell *rightShardIntervalCell = NULL;
ShardInterval *previousRightInterval = NULL;
/* we do not support outer join queries on tables with no shards */
if (leftShardIntervalCount == 0 || rightShardIntervalCount == 0)
{
return false;
}
if (leftShardIntervalCount != rightShardIntervalCount)
{
return false;
}
forboth(leftShardIntervalCell, leftShardIntervalList,
rightShardIntervalCell, rightShardIntervalList)
{
ShardInterval *leftInterval = (ShardInterval *) lfirst(leftShardIntervalCell);
ShardInterval *rightInterval = (ShardInterval *) lfirst(rightShardIntervalCell);
ListCell *nextRightIntervalCell = NULL;
bool shardIntervalsIntersect = ShardIntervalsOverlap(leftInterval, rightInterval);
if (!shardIntervalsIntersect)
{
return false;
}
/*
* Compare left interval with a previous right interval, they should not
* intersect.
*/
if (previousRightInterval != NULL)
{
shardIntervalsIntersect = ShardIntervalsOverlap(leftInterval,
previousRightInterval);
if (shardIntervalsIntersect)
{
return false;
}
}
/*
* Compare left interval with a next right interval, they should not
* intersect.
*/
nextRightIntervalCell = lnext(rightShardIntervalCell);
if (nextRightIntervalCell != NULL)
{
ShardInterval *nextRightInterval =
(ShardInterval *) lfirst(nextRightIntervalCell);
shardIntervalsIntersect = ShardIntervalsOverlap(leftInterval,
nextRightInterval);
if (shardIntervalsIntersect)
{
return false;
}
}
previousRightInterval = rightInterval;
}
return true;
}
/*
* JoinOrderList calculates the best join order and join rules that apply given
* the list of tables and join clauses. First, the function generates a set of
@ -703,12 +478,10 @@ JoinOrderForTable(TableEntry *firstTable, List *tableEntryList, List *joinClause
JoinOrderNode *pendingJoinNode = NULL;
JoinRuleType pendingJoinRuleType = JOIN_RULE_LAST;
JoinType joinType = JOIN_INNER;
List *candidateShardList = LoadShardIntervalList(pendingTable->relationId);
/* evaluate all join rules for this pending table */
pendingJoinNode = EvaluateJoinRules(joinedTableList, currentJoinNode,
pendingTable, candidateShardList,
joinClauseList, joinType);
pendingTable, joinClauseList, joinType);
/* if this rule is better than previous ones, keep it */
pendingJoinRuleType = pendingJoinNode->joinRuleType;
@ -1010,8 +783,8 @@ FindTableEntry(List *tableEntryList, uint32 tableId)
*/
static JoinOrderNode *
EvaluateJoinRules(List *joinedTableList, JoinOrderNode *currentJoinNode,
TableEntry *candidateTable, List *candidateShardList,
List *joinClauseList, JoinType joinType)
TableEntry *candidateTable, List *joinClauseList,
JoinType joinType)
{
JoinOrderNode *nextJoinNode = NULL;
uint32 candidateTableId = 0;
@ -1038,7 +811,6 @@ EvaluateJoinRules(List *joinedTableList, JoinOrderNode *currentJoinNode,
nextJoinNode = (*ruleEvalFunction)(currentJoinNode,
candidateTable,
candidateShardList,
applicableJoinClauses,
joinType);
@ -1089,7 +861,7 @@ JoinRuleEvalFunction(JoinRuleType ruleType)
if (!ruleEvalFunctionsInitialized)
{
RuleEvalFunctionArray[BROADCAST_JOIN] = &BroadcastJoin;
RuleEvalFunctionArray[REFERENCE_JOIN] = &ReferenceJoin;
RuleEvalFunctionArray[LOCAL_PARTITION_JOIN] = &LocalJoin;
RuleEvalFunctionArray[SINGLE_PARTITION_JOIN] = &SinglePartitionJoin;
RuleEvalFunctionArray[DUAL_PARTITION_JOIN] = &DualPartitionJoin;
@ -1115,7 +887,7 @@ JoinRuleName(JoinRuleType ruleType)
if (!ruleNamesInitialized)
{
/* use strdup() to be independent of memory contexts */
RuleNameArray[BROADCAST_JOIN] = strdup("broadcast join");
RuleNameArray[REFERENCE_JOIN] = strdup("reference join");
RuleNameArray[LOCAL_PARTITION_JOIN] = strdup("local partition join");
RuleNameArray[SINGLE_PARTITION_JOIN] = strdup("single partition join");
RuleNameArray[DUAL_PARTITION_JOIN] = strdup("dual partition join");
@ -1132,21 +904,19 @@ JoinRuleName(JoinRuleType ruleType)
/*
* BroadcastJoin evaluates if the candidate table is small enough to be
* broadcasted to all nodes in the system. If the table can be broadcasted,
* the function simply returns a join order node that includes the current
* partition key and method. Otherwise, the function returns null.
* ReferenceJoin evaluates if the candidate table is a reference table for inner,
* left and anti join. For right join, current join node must be represented by
* a reference table. For full join, both of them must be a reference table.
*/
static JoinOrderNode *
BroadcastJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
List *candidateShardList, List *applicableJoinClauses,
JoinType joinType)
ReferenceJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
List *applicableJoinClauses, JoinType joinType)
{
JoinOrderNode *nextJoinNode = NULL;
int candidateShardCount = list_length(candidateShardList);
int leftShardCount = list_length(currentJoinNode->shardIntervalList);
int applicableJoinCount = list_length(applicableJoinClauses);
bool performBroadcastJoin = false;
char candidatePartitionMethod = PartitionMethod(candidateTable->relationId);
char leftPartitionMethod = PartitionMethod(currentJoinNode->tableEntry->relationId);
bool performReferenceJoin = false;
if (applicableJoinCount <= 0)
{
@ -1154,49 +924,30 @@ BroadcastJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
}
/*
* If the table's shard count doesn't exceed the value specified in the
* configuration or the table is a reference table, then we assume table
* broadcasting is feasible. This assumption is valid only for inner joins.
* If the table is a reference table, then the reference join is feasible.It
* is valid only for inner joins.
*
* Left join requires candidate table to have single shard, right join requires
* existing (left) table to have single shard, full outer join requires both tables
* to have single shard.
* Right join requires existing (left) table to be reference table, full outer
* join requires both tables to be reference tables.
*/
if (joinType == JOIN_INNER)
if ((joinType == JOIN_INNER || joinType == JOIN_LEFT || joinType == JOIN_ANTI) &&
candidatePartitionMethod == DISTRIBUTE_BY_NONE)
{
ShardInterval *initialCandidateShardInterval = NULL;
char candidatePartitionMethod = '\0';
if (candidateShardCount > 0)
{
initialCandidateShardInterval =
(ShardInterval *) linitial(candidateShardList);
candidatePartitionMethod =
PartitionMethod(initialCandidateShardInterval->relationId);
}
if (candidatePartitionMethod == DISTRIBUTE_BY_NONE ||
candidateShardCount < LargeTableShardCount)
{
performBroadcastJoin = true;
}
performReferenceJoin = true;
}
else if ((joinType == JOIN_LEFT || joinType == JOIN_ANTI) && candidateShardCount == 1)
else if (joinType == JOIN_RIGHT && leftPartitionMethod == DISTRIBUTE_BY_NONE)
{
performBroadcastJoin = true;
performReferenceJoin = true;
}
else if (joinType == JOIN_RIGHT && leftShardCount == 1)
else if (joinType == JOIN_FULL && leftPartitionMethod == DISTRIBUTE_BY_NONE &&
candidatePartitionMethod == DISTRIBUTE_BY_NONE)
{
performBroadcastJoin = true;
}
else if (joinType == JOIN_FULL && leftShardCount == 1 && candidateShardCount == 1)
{
performBroadcastJoin = true;
performReferenceJoin = true;
}
if (performBroadcastJoin)
if (performReferenceJoin)
{
nextJoinNode = MakeJoinOrderNode(candidateTable, BROADCAST_JOIN,
nextJoinNode = MakeJoinOrderNode(candidateTable, REFERENCE_JOIN,
currentJoinNode->partitionColumn,
currentJoinNode->partitionMethod,
currentJoinNode->anchorTable);
@ -1220,8 +971,7 @@ BroadcastJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
*/
static JoinOrderNode *
LocalJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
List *candidateShardList, List *applicableJoinClauses,
JoinType joinType)
List *applicableJoinClauses, JoinType joinType)
{
JoinOrderNode *nextJoinNode = NULL;
Oid relationId = candidateTable->relationId;
@ -1288,8 +1038,7 @@ LocalJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
*/
static JoinOrderNode *
SinglePartitionJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
List *candidateShardList, List *applicableJoinClauses,
JoinType joinType)
List *applicableJoinClauses, JoinType joinType)
{
JoinOrderNode *nextJoinNode = NULL;
Var *currentPartitionColumn = currentJoinNode->partitionColumn;
@ -1400,8 +1149,7 @@ SinglePartitionJoinClause(Var *partitionColumn, List *applicableJoinClauses)
*/
static JoinOrderNode *
DualPartitionJoin(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
List *candidateShardList, List *applicableJoinClauses,
JoinType joinType)
List *applicableJoinClauses, JoinType joinType)
{
/* Because of the dual partition, anchor table information got lost */
TableEntry *anchorTable = NULL;
@ -1460,8 +1208,7 @@ DualPartitionJoinClause(List *applicableJoinClauses)
*/
static JoinOrderNode *
CartesianProduct(JoinOrderNode *currentJoinNode, TableEntry *candidateTable,
List *candidateShardList, List *applicableJoinClauses,
JoinType joinType)
List *applicableJoinClauses, JoinType joinType)
{
/* Because of the cartesian product, anchor table information got lost */
TableEntry *anchorTable = NULL;

View File

@ -131,7 +131,7 @@ static MultiNode * ApplyJoinRule(MultiNode *leftNode, MultiNode *rightNode,
JoinRuleType ruleType, Var *partitionColumn,
JoinType joinType, List *joinClauseList);
static RuleApplyFunction JoinRuleApplyFunction(JoinRuleType ruleType);
static MultiNode * ApplyBroadcastJoin(MultiNode *leftNode, MultiNode *rightNode,
static MultiNode * ApplyReferenceJoin(MultiNode *leftNode, MultiNode *rightNode,
Var *partitionColumn, JoinType joinType,
List *joinClauses);
static MultiNode * ApplyLocalJoin(MultiNode *leftNode, MultiNode *rightNode,
@ -3473,7 +3473,7 @@ JoinRuleApplyFunction(JoinRuleType ruleType)
if (!ruleApplyFunctionInitialized)
{
RuleApplyFunctionArray[BROADCAST_JOIN] = &ApplyBroadcastJoin;
RuleApplyFunctionArray[REFERENCE_JOIN] = &ApplyReferenceJoin;
RuleApplyFunctionArray[LOCAL_PARTITION_JOIN] = &ApplyLocalJoin;
RuleApplyFunctionArray[SINGLE_PARTITION_JOIN] = &ApplySinglePartitionJoin;
RuleApplyFunctionArray[DUAL_PARTITION_JOIN] = &ApplyDualPartitionJoin;
@ -3494,12 +3494,12 @@ JoinRuleApplyFunction(JoinRuleType ruleType)
* right node. The new node uses the broadcast join rule to perform the join.
*/
static MultiNode *
ApplyBroadcastJoin(MultiNode *leftNode, MultiNode *rightNode,
ApplyReferenceJoin(MultiNode *leftNode, MultiNode *rightNode,
Var *partitionColumn, JoinType joinType,
List *applicableJoinClauses)
{
MultiJoin *joinNode = CitusMakeNode(MultiJoin);
joinNode->joinRuleType = BROADCAST_JOIN;
joinNode->joinRuleType = REFERENCE_JOIN;
joinNode->joinType = joinType;
joinNode->joinClauseList = applicableJoinClauses;

View File

@ -28,7 +28,7 @@
typedef enum JoinRuleType
{
JOIN_RULE_INVALID_FIRST = 0,
BROADCAST_JOIN = 1,
REFERENCE_JOIN = 1,
LOCAL_PARTITION_JOIN = 2,
SINGLE_PARTITION_JOIN = 3,
DUAL_PARTITION_JOIN = 4,
@ -69,7 +69,6 @@ typedef struct JoinOrderNode
Var *partitionColumn; /* not relevant for the first table */
char partitionMethod;
List *joinClauseList; /* not relevant for the first table */
List *shardIntervalList;
TableEntry *anchorTable;
} JoinOrderNode;

View File

@ -4,7 +4,9 @@
ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 360000;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 100000;
-- Create new table definitions for use in testing in distributed planning and
-- execution functionality. Also create indexes to boost performance.
-- execution functionality. Also create indexes to boost performance. Since we
-- need to cover both reference join and partitioned join, we have created
-- reference and append distributed version of orders, customer and part tables.
CREATE TABLE lineitem (
l_orderkey bigint not null,
l_partkey integer not null,
@ -53,6 +55,23 @@ HINT: Consider using hash partitioning.
(1 row)
CREATE TABLE orders_reference (
o_orderkey bigint not null,
o_custkey integer not null,
o_orderstatus char(1) not null,
o_totalprice decimal(15,2) not null,
o_orderdate date not null,
o_orderpriority char(15) not null,
o_clerk char(15) not null,
o_shippriority integer not null,
o_comment varchar(79) not null,
PRIMARY KEY(o_orderkey) );
SELECT create_reference_table('orders_reference');
create_reference_table
------------------------
(1 row)
CREATE TABLE customer (
c_custkey integer not null,
c_name varchar(25) not null,
@ -62,7 +81,22 @@ CREATE TABLE customer (
c_acctbal decimal(15,2) not null,
c_mktsegment char(10) not null,
c_comment varchar(117) not null);
SELECT master_create_distributed_table('customer', 'c_custkey', 'append');
SELECT create_reference_table('customer');
create_reference_table
------------------------
(1 row)
CREATE TABLE customer_append (
c_custkey integer not null,
c_name varchar(25) not null,
c_address varchar(40) not null,
c_nationkey integer not null,
c_phone char(15) not null,
c_acctbal decimal(15,2) not null,
c_mktsegment char(10) not null,
c_comment varchar(117) not null);
SELECT master_create_distributed_table('customer_append', 'c_custkey', 'append');
master_create_distributed_table
---------------------------------
@ -89,7 +123,23 @@ CREATE TABLE part (
p_container char(10) not null,
p_retailprice decimal(15,2) not null,
p_comment varchar(23) not null);
SELECT master_create_distributed_table('part', 'p_partkey', 'append');
SELECT create_reference_table('part');
create_reference_table
------------------------
(1 row)
CREATE TABLE part_append (
p_partkey integer not null,
p_name varchar(55) not null,
p_mfgr char(25) not null,
p_brand char(10) not null,
p_type varchar(25) not null,
p_size integer not null,
p_container char(10) not null,
p_retailprice decimal(15,2) not null,
p_comment varchar(23) not null);
SELECT master_create_distributed_table('part_append', 'p_partkey', 'append');
master_create_distributed_table
---------------------------------
@ -695,13 +745,13 @@ INSERT INTO tt2 SELECT * FROM tt1 WHERE id = 1;
COMMIT;
-- Table should exist on the worker node
\c - - - :worker_1_port
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360066'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360069'::regclass;
Column | Type | Modifiers
--------+---------+-----------
id | integer |
(1 row)
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt2_360070'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt2_360073'::regclass;
Column | Type | Modifiers
--------+---------+-----------
id | integer |
@ -723,13 +773,13 @@ SELECT create_distributed_table('append_tt1','id','append');
SELECT master_create_empty_shard('append_tt1');
master_create_empty_shard
---------------------------
360074
360077
(1 row)
ROLLBACK;
-- Table exists on the worker node.
\c - - - :worker_1_port
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.append_tt1_360074'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.append_tt1_360077'::regclass;
Column | Type | Modifiers
--------+---------+-----------
id | integer |
@ -766,7 +816,7 @@ SELECT * FROM tt1 WHERE id = 1;
COMMIT;
-- Placements should be created on the worker
\c - - - :worker_1_port
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360075'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360078'::regclass;
Column | Type | Modifiers
--------+---------+-----------
id | integer |

View File

@ -762,20 +762,20 @@ Custom Scan (Citus Router)
Tasks Shown: All
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360039 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360039 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360040 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360040 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360041 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360042 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360042 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360043 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360043 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360044 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360044 lineitem_hash_part
EXPLAIN (COSTS FALSE)
UPDATE lineitem_hash_part
@ -786,13 +786,13 @@ Custom Scan (Citus Router)
Tasks Shown: All
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Update on lineitem_hash_part_360041 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
Filter: ((l_orderkey = 1) OR (l_orderkey = 3))
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360039 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360039 lineitem_hash_part
-> Update on lineitem_hash_part_360042 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360042 lineitem_hash_part
Filter: ((l_orderkey = 1) OR (l_orderkey = 3))
-- Test multi shard delete
EXPLAIN (COSTS FALSE)
@ -802,20 +802,20 @@ Custom Scan (Citus Router)
Tasks Shown: All
-> Task
Node: host=localhost port=57637 dbname=regression
-> Delete on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360039 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360039 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Delete on lineitem_hash_part_360040 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360040 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360041 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360042 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360042 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Delete on lineitem_hash_part_360043 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360043 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360044 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360044 lineitem_hash_part
-- Test track tracker
SET citus.task_executor_type TO 'task-tracker';
SET citus.explain_all_tasks TO off;
@ -834,7 +834,7 @@ Aggregate
SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
@ -850,7 +850,7 @@ Aggregate
Merge Task Count: 1
EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
@ -892,14 +892,14 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
]
SELECT true AS valid FROM explain_json($$
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey$$);
t
EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
@ -941,7 +941,7 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</explain>
SELECT true AS valid FROM explain_xml($$
SELECT count(*)
FROM lineitem, orders, customer, supplier
FROM lineitem, orders, customer_append, supplier
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey$$);
@ -980,11 +980,8 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Task Count: 1
Tasks Shown: "None, not supported for re-partition queries"
Depended Jobs:
- Map Task Count: 1
- Map Task Count: 2
Merge Task Count: 1
Depended Jobs:
- Map Task Count: 2
Merge Task Count: 1
-- test parallel aggregates
SET parallel_setup_cost=0;
SET parallel_tuple_cost=0;
@ -1075,7 +1072,7 @@ Custom Scan (Citus INSERT ... SELECT via coordinator)
-> Task
Node: host=localhost port=57637 dbname=regression
-> Limit
-> Seq Scan on orders_hash_part_360043 orders_hash_part
-> Seq Scan on orders_hash_part_360046 orders_hash_part
SELECT true AS valid FROM explain_json($$
INSERT INTO lineitem_hash_part (l_orderkey)
SELECT o_orderkey FROM orders_hash_part LIMIT 3;
@ -1092,7 +1089,7 @@ Custom Scan (Citus INSERT ... SELECT via coordinator)
-> Task
Node: host=localhost port=57637 dbname=regression
-> Limit
-> Seq Scan on orders_hash_part_360043 orders_hash_part
-> Seq Scan on orders_hash_part_360046 orders_hash_part
EXPLAIN (COSTS OFF)
INSERT INTO lineitem_hash_part (l_orderkey)
SELECT s FROM generate_series(1,5) s;
@ -1145,7 +1142,7 @@ Custom Scan (Citus Router)
-> HashAggregate
Output: l_orderkey
Group Key: lineitem_hash_part.l_orderkey
-> Seq Scan on public.lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on public.lineitem_hash_part_360041 lineitem_hash_part
Output: 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
-> Distributed Subplan 55_2
-> Function Scan on pg_catalog.generate_series s

View File

@ -762,20 +762,20 @@ Custom Scan (Citus Router)
Tasks Shown: All
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360039 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360039 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360040 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360040 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360041 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360042 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360042 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360043 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360043 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360044 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360044 lineitem_hash_part
EXPLAIN (COSTS FALSE)
UPDATE lineitem_hash_part
@ -786,13 +786,13 @@ Custom Scan (Citus Router)
Tasks Shown: All
-> Task
Node: host=localhost port=57637 dbname=regression
-> Update on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Update on lineitem_hash_part_360041 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
Filter: ((l_orderkey = 1) OR (l_orderkey = 3))
-> Task
Node: host=localhost port=57638 dbname=regression
-> Update on lineitem_hash_part_360039 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360039 lineitem_hash_part
-> Update on lineitem_hash_part_360042 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360042 lineitem_hash_part
Filter: ((l_orderkey = 1) OR (l_orderkey = 3))
-- Test multi shard delete
EXPLAIN (COSTS FALSE)
@ -802,20 +802,20 @@ Custom Scan (Citus Router)
Tasks Shown: All
-> Task
Node: host=localhost port=57637 dbname=regression
-> Delete on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360039 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360039 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Delete on lineitem_hash_part_360040 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360040 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360041 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360042 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360042 lineitem_hash_part
-> Task
Node: host=localhost port=57637 dbname=regression
-> Delete on lineitem_hash_part_360043 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360043 lineitem_hash_part
-> Task
Node: host=localhost port=57638 dbname=regression
-> Delete on lineitem_hash_part_360044 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360044 lineitem_hash_part
-- Test track tracker
SET citus.task_executor_type TO 'task-tracker';
SET citus.explain_all_tasks TO off;
@ -834,7 +834,7 @@ Aggregate
SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
@ -850,7 +850,7 @@ Aggregate
Merge Task Count: 1
EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
@ -892,14 +892,14 @@ EXPLAIN (COSTS FALSE, FORMAT JSON)
]
SELECT true AS valid FROM explain_json($$
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey$$);
t
EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
@ -941,7 +941,7 @@ EXPLAIN (COSTS FALSE, FORMAT XML)
</explain>
SELECT true AS valid FROM explain_xml($$
SELECT count(*)
FROM lineitem, orders, customer, supplier
FROM lineitem, orders, customer_append, supplier
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey$$);
@ -980,11 +980,8 @@ EXPLAIN (COSTS FALSE, FORMAT YAML)
Task Count: 1
Tasks Shown: "None, not supported for re-partition queries"
Depended Jobs:
- Map Task Count: 1
- Map Task Count: 2
Merge Task Count: 1
Depended Jobs:
- Map Task Count: 2
Merge Task Count: 1
-- test parallel aggregates
SET parallel_setup_cost=0;
SET parallel_tuple_cost=0;
@ -1075,7 +1072,7 @@ Custom Scan (Citus INSERT ... SELECT via coordinator)
-> Task
Node: host=localhost port=57637 dbname=regression
-> Limit
-> Seq Scan on orders_hash_part_360043 orders_hash_part
-> Seq Scan on orders_hash_part_360046 orders_hash_part
SELECT true AS valid FROM explain_json($$
INSERT INTO lineitem_hash_part (l_orderkey)
SELECT o_orderkey FROM orders_hash_part LIMIT 3;
@ -1092,7 +1089,7 @@ Custom Scan (Citus INSERT ... SELECT via coordinator)
-> Task
Node: host=localhost port=57637 dbname=regression
-> Limit
-> Seq Scan on orders_hash_part_360043 orders_hash_part
-> Seq Scan on orders_hash_part_360046 orders_hash_part
EXPLAIN (COSTS OFF)
INSERT INTO lineitem_hash_part (l_orderkey)
SELECT s FROM generate_series(1,5) s;
@ -1145,7 +1142,7 @@ Custom Scan (Citus Router)
-> HashAggregate
Output: l_orderkey
Group Key: lineitem_hash_part.l_orderkey
-> Seq Scan on public.lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on public.lineitem_hash_part_360041 lineitem_hash_part
Output: 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
-> Distributed Subplan 55_2
-> Function Scan on pg_catalog.generate_series s

View File

@ -76,7 +76,7 @@ SELECT create_distributed_table('customer_hash', 'c_custkey');
-- The following query checks that we can correctly handle self-joins
EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
LOG: join order: [ "lineitem" ][ broadcast join "lineitem" ]
LOG: join order: [ "lineitem" ][ local partition join "lineitem" ]
DEBUG: join prunable for intervals [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
QUERY PLAN
@ -137,7 +137,7 @@ LOG: join order: [ "orders_hash" ][ local partition join "lineitem_hash" ]
-- Validate that we can handle broadcast joins with hash-partitioned tables.
EXPLAIN SELECT count(*) FROM customer_hash, nation
WHERE c_nationkey = n_nationkey;
LOG: join order: [ "customer_hash" ][ broadcast join "nation" ]
LOG: join order: [ "customer_hash" ][ reference join "nation" ]
QUERY PLAN
--------------------------------------------------------------------------
Aggregate (cost=0.00..0.00 rows=0 width=0)
@ -149,9 +149,9 @@ LOG: join order: [ "customer_hash" ][ broadcast join "nation" ]
SET citus.large_table_shard_count TO 1;
-- 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.
EXPLAIN SELECT count(*) FROM orders, lineitem, customer
EXPLAIN SELECT count(*) FROM orders, lineitem, customer_append
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer" ]
LOG: join order: [ "orders" ][ dual partition join "lineitem" ][ dual partition join "customer_append" ]
QUERY PLAN
--------------------------------------------------------------------------
Aggregate (cost=0.00..0.00 rows=0 width=0)
@ -173,9 +173,9 @@ LOG: join order: [ "orders" ][ dual partition join "customer_hash" ]
-- Validate that we can re-partition a hash partitioned table to join with a
-- range partitioned one.
EXPLAIN SELECT count(*) FROM orders_hash, customer
EXPLAIN SELECT count(*) FROM orders_hash, customer_append
WHERE c_custkey = o_custkey;
LOG: join order: [ "orders_hash" ][ single partition join "customer" ]
LOG: join order: [ "orders_hash" ][ single partition join "customer_append" ]
QUERY PLAN
--------------------------------------------------------------------------
Aggregate (cost=0.00..0.00 rows=0 width=0)

View File

@ -37,7 +37,7 @@ EXPLAIN SELECT
o_orderdate,
o_shippriority
FROM
customer,
customer_append,
orders,
lineitem
WHERE
@ -53,7 +53,7 @@ GROUP BY
ORDER BY
revenue DESC,
o_orderdate;
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ]
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer_append" ]
QUERY PLAN
------------------------------------------------------------------------------------------------
Sort (cost=0.00..0.00 rows=0 width=0)
@ -75,7 +75,7 @@ EXPLAIN SELECT
c_phone,
c_comment
FROM
customer,
customer_append,
orders,
lineitem,
nation
@ -96,7 +96,7 @@ GROUP BY
c_comment
ORDER BY
revenue DESC;
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer" ][ broadcast join "nation" ]
LOG: join order: [ "orders" ][ local partition join "lineitem" ][ single partition join "customer_append" ][ reference join "nation" ]
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Sort (cost=0.00..0.00 rows=0 width=0)
@ -112,7 +112,7 @@ EXPLAIN SELECT
sum(l_extendedprice* (1 - l_discount)) as revenue
FROM
lineitem,
part
part_append
WHERE
(
p_partkey = l_partkey
@ -137,7 +137,7 @@ WHERE
AND l_shipmode in ('AIR', 'AIR REG', 'TRUCK')
AND l_shipinstruct = 'DELIVER IN PERSON'
);
LOG: join order: [ "lineitem" ][ single partition join "part" ]
LOG: join order: [ "lineitem" ][ single partition join "part_append" ]
QUERY PLAN
--------------------------------------------------------------------------
Aggregate (cost=0.00..0.00 rows=0 width=0)
@ -149,14 +149,14 @@ LOG: join order: [ "lineitem" ][ single partition join "part" ]
EXPLAIN SELECT
l_partkey, count(*)
FROM
lineitem, part, orders, customer
lineitem, part_append, orders, customer_append
WHERE
l_orderkey = o_orderkey AND
l_partkey = p_partkey AND
c_custkey = o_custkey
GROUP BY
l_partkey;
LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single partition join "part" ][ single partition join "customer" ]
LOG: join order: [ "lineitem" ][ local partition join "orders" ][ single partition join "part_append" ][ single partition join "customer_append" ]
QUERY PLAN
--------------------------------------------------------------------------
HashAggregate (cost=0.00..0.00 rows=0 width=0)

View File

@ -48,7 +48,7 @@ GROUP BY
ORDER BY
revenue DESC,
o_orderdate;
LOG: join order: [ "orders" ][ broadcast join "customer" ][ local partition join "lineitem" ]
LOG: join order: [ "orders" ][ reference join "customer" ][ local partition join "lineitem" ]
QUERY PLAN
------------------------------------------------------------------------------------------------
Sort (cost=0.00..0.00 rows=0 width=0)
@ -91,7 +91,7 @@ GROUP BY
c_comment
ORDER BY
revenue DESC;
LOG: join order: [ "orders" ][ broadcast join "customer" ][ broadcast join "nation" ][ local partition join "lineitem" ]
LOG: join order: [ "orders" ][ reference join "customer" ][ reference join "nation" ][ local partition join "lineitem" ]
QUERY PLAN
----------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Sort (cost=0.00..0.00 rows=0 width=0)
@ -132,7 +132,7 @@ WHERE
AND l_shipmode in ('AIR', 'AIR REG', 'TRUCK')
AND l_shipinstruct = 'DELIVER IN PERSON'
);
LOG: join order: [ "lineitem" ][ broadcast join "part" ]
LOG: join order: [ "lineitem" ][ reference join "part" ]
QUERY PLAN
-----------------------------------------------------------------------
Aggregate (cost=0.00..0.00 rows=0 width=0)

View File

@ -29,7 +29,7 @@ SET citus.task_executor_type TO 'task-tracker';
SELECT
l_partkey, o_orderkey, count(*)
FROM
lineitem, part, orders, customer
lineitem, part_append, orders, customer_append
WHERE
l_orderkey = o_orderkey AND
l_partkey = p_partkey AND
@ -52,9 +52,9 @@ DEBUG: assigned task 1 to node localhost:57638
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
DEBUG: generated sql query for task 2
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000003".intermediate_column_1_0, "pg_merge_job_0001.task_000003".intermediate_column_1_1, "pg_merge_job_0001.task_000003".intermediate_column_1_2, "pg_merge_job_0001.task_000003".intermediate_column_1_3, "pg_merge_job_0001.task_000003".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000003 "pg_merge_job_0001.task_000003" JOIN part_290005 part ON (("pg_merge_job_0001.task_000003".intermediate_column_1_0 = part.p_partkey))) WHERE (part.p_size > 8)"
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000003".intermediate_column_1_0, "pg_merge_job_0001.task_000003".intermediate_column_1_1, "pg_merge_job_0001.task_000003".intermediate_column_1_2, "pg_merge_job_0001.task_000003".intermediate_column_1_3, "pg_merge_job_0001.task_000003".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000003 "pg_merge_job_0001.task_000003" JOIN part_append_290005 part_append ON (("pg_merge_job_0001.task_000003".intermediate_column_1_0 = part_append.p_partkey))) WHERE (part_append.p_size > 8)"
DEBUG: generated sql query for task 4
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000006".intermediate_column_1_0, "pg_merge_job_0001.task_000006".intermediate_column_1_1, "pg_merge_job_0001.task_000006".intermediate_column_1_2, "pg_merge_job_0001.task_000006".intermediate_column_1_3, "pg_merge_job_0001.task_000006".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000006 "pg_merge_job_0001.task_000006" JOIN part_280002 part ON (("pg_merge_job_0001.task_000006".intermediate_column_1_0 = part.p_partkey))) WHERE (part.p_size > 8)"
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000006".intermediate_column_1_0, "pg_merge_job_0001.task_000006".intermediate_column_1_1, "pg_merge_job_0001.task_000006".intermediate_column_1_2, "pg_merge_job_0001.task_000006".intermediate_column_1_3, "pg_merge_job_0001.task_000006".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000006 "pg_merge_job_0001.task_000006" JOIN part_append_280002 part_append ON (("pg_merge_job_0001.task_000006".intermediate_column_1_0 = part_append.p_partkey))) WHERE (part_append.p_size > 8)"
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 3
DEBUG: pruning merge fetch taskId 3
@ -68,11 +68,11 @@ DEBUG: join prunable for intervals [1001,2000] and [6001,7000]
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
DEBUG: generated sql query for task 2
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000005".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000005".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000005 "pg_merge_job_0002.task_000005" JOIN customer_290004 customer ON ((customer.c_custkey = "pg_merge_job_0002.task_000005".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000005".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000005".intermediate_column_2_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000005".intermediate_column_2_0, "pg_merge_job_0002.task_000005".intermediate_column_2_1"
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000005".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000005".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000005 "pg_merge_job_0002.task_000005" JOIN customer_append_290004 customer_append ON ((customer_append.c_custkey = "pg_merge_job_0002.task_000005".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000005".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000005".intermediate_column_2_3 > 1200.0)) AND (customer_append.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000005".intermediate_column_2_0, "pg_merge_job_0002.task_000005".intermediate_column_2_1"
DEBUG: generated sql query for task 4
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000008".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000008".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000008 "pg_merge_job_0002.task_000008" JOIN customer_280001 customer ON ((customer.c_custkey = "pg_merge_job_0002.task_000008".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000008".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000008".intermediate_column_2_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000008".intermediate_column_2_0, "pg_merge_job_0002.task_000008".intermediate_column_2_1"
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000008".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000008".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000008 "pg_merge_job_0002.task_000008" JOIN customer_append_280001 customer_append ON ((customer_append.c_custkey = "pg_merge_job_0002.task_000008".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000008".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000008".intermediate_column_2_3 > 1200.0)) AND (customer_append.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000008".intermediate_column_2_0, "pg_merge_job_0002.task_000008".intermediate_column_2_1"
DEBUG: generated sql query for task 6
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000011".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000011".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000011 "pg_merge_job_0002.task_000011" JOIN customer_280000 customer ON ((customer.c_custkey = "pg_merge_job_0002.task_000011".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000011".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000011".intermediate_column_2_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000011".intermediate_column_2_0, "pg_merge_job_0002.task_000011".intermediate_column_2_1"
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000011".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000011".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000011 "pg_merge_job_0002.task_000011" JOIN customer_append_280000 customer_append ON ((customer_append.c_custkey = "pg_merge_job_0002.task_000011".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000011".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000011".intermediate_column_2_3 > 1200.0)) AND (customer_append.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000011".intermediate_column_2_0, "pg_merge_job_0002.task_000011".intermediate_column_2_1"
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 5
DEBUG: pruning merge fetch taskId 3

View File

@ -36,7 +36,7 @@ DEBUG: CommitTransactionCommand
SELECT
l_partkey, o_orderkey, count(*)
FROM
lineitem, part, orders, customer
lineitem, part_append, orders, customer_append
WHERE
l_orderkey = o_orderkey AND
l_partkey = p_partkey AND
@ -60,9 +60,9 @@ DEBUG: assigned task 1 to node localhost:57638
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
DEBUG: generated sql query for task 2
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000003".intermediate_column_1_0, "pg_merge_job_0001.task_000003".intermediate_column_1_1, "pg_merge_job_0001.task_000003".intermediate_column_1_2, "pg_merge_job_0001.task_000003".intermediate_column_1_3, "pg_merge_job_0001.task_000003".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000003 "pg_merge_job_0001.task_000003" JOIN part_290005 part ON (("pg_merge_job_0001.task_000003".intermediate_column_1_0 = part.p_partkey))) WHERE (part.p_size > 8)"
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000003".intermediate_column_1_0, "pg_merge_job_0001.task_000003".intermediate_column_1_1, "pg_merge_job_0001.task_000003".intermediate_column_1_2, "pg_merge_job_0001.task_000003".intermediate_column_1_3, "pg_merge_job_0001.task_000003".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000003 "pg_merge_job_0001.task_000003" JOIN part_append_290005 part_append ON (("pg_merge_job_0001.task_000003".intermediate_column_1_0 = part_append.p_partkey))) WHERE (part_append.p_size > 8)"
DEBUG: generated sql query for task 4
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000006".intermediate_column_1_0, "pg_merge_job_0001.task_000006".intermediate_column_1_1, "pg_merge_job_0001.task_000006".intermediate_column_1_2, "pg_merge_job_0001.task_000006".intermediate_column_1_3, "pg_merge_job_0001.task_000006".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000006 "pg_merge_job_0001.task_000006" JOIN part_280002 part ON (("pg_merge_job_0001.task_000006".intermediate_column_1_0 = part.p_partkey))) WHERE (part.p_size > 8)"
DETAIL: query string: "SELECT "pg_merge_job_0001.task_000006".intermediate_column_1_0, "pg_merge_job_0001.task_000006".intermediate_column_1_1, "pg_merge_job_0001.task_000006".intermediate_column_1_2, "pg_merge_job_0001.task_000006".intermediate_column_1_3, "pg_merge_job_0001.task_000006".intermediate_column_1_4 FROM (pg_merge_job_0001.task_000006 "pg_merge_job_0001.task_000006" JOIN part_append_280002 part_append ON (("pg_merge_job_0001.task_000006".intermediate_column_1_0 = part_append.p_partkey))) WHERE (part_append.p_size > 8)"
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 3
DEBUG: pruning merge fetch taskId 3
@ -76,11 +76,11 @@ DEBUG: join prunable for intervals [1001,2000] and [6001,7000]
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
DEBUG: generated sql query for task 2
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000005".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000005".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000005 "pg_merge_job_0002.task_000005" JOIN customer_290004 customer ON ((customer.c_custkey = "pg_merge_job_0002.task_000005".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000005".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000005".intermediate_column_2_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000005".intermediate_column_2_0, "pg_merge_job_0002.task_000005".intermediate_column_2_1"
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000005".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000005".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000005 "pg_merge_job_0002.task_000005" JOIN customer_append_290004 customer_append ON ((customer_append.c_custkey = "pg_merge_job_0002.task_000005".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000005".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000005".intermediate_column_2_3 > 1200.0)) AND (customer_append.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000005".intermediate_column_2_0, "pg_merge_job_0002.task_000005".intermediate_column_2_1"
DEBUG: generated sql query for task 4
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000008".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000008".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000008 "pg_merge_job_0002.task_000008" JOIN customer_280001 customer ON ((customer.c_custkey = "pg_merge_job_0002.task_000008".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000008".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000008".intermediate_column_2_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000008".intermediate_column_2_0, "pg_merge_job_0002.task_000008".intermediate_column_2_1"
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000008".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000008".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000008 "pg_merge_job_0002.task_000008" JOIN customer_append_280001 customer_append ON ((customer_append.c_custkey = "pg_merge_job_0002.task_000008".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000008".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000008".intermediate_column_2_3 > 1200.0)) AND (customer_append.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000008".intermediate_column_2_0, "pg_merge_job_0002.task_000008".intermediate_column_2_1"
DEBUG: generated sql query for task 6
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000011".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000011".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000011 "pg_merge_job_0002.task_000011" JOIN customer_280000 customer ON ((customer.c_custkey = "pg_merge_job_0002.task_000011".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000011".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000011".intermediate_column_2_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000011".intermediate_column_2_0, "pg_merge_job_0002.task_000011".intermediate_column_2_1"
DETAIL: query string: "SELECT "pg_merge_job_0002.task_000011".intermediate_column_2_0 AS l_partkey, "pg_merge_job_0002.task_000011".intermediate_column_2_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_0002.task_000011 "pg_merge_job_0002.task_000011" JOIN customer_append_280000 customer_append ON ((customer_append.c_custkey = "pg_merge_job_0002.task_000011".intermediate_column_2_4))) WHERE ((("pg_merge_job_0002.task_000011".intermediate_column_2_2 > 5.0) OR ("pg_merge_job_0002.task_000011".intermediate_column_2_3 > 1200.0)) AND (customer_append.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_0002.task_000011".intermediate_column_2_0, "pg_merge_job_0002.task_000011".intermediate_column_2_1"
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 5
DEBUG: pruning merge fetch taskId 3

View File

@ -13,7 +13,7 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
@ -42,7 +42,7 @@ DETAIL: Creating dependency on merge taskId 9
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
@ -68,7 +68,7 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
o_orderkey < 0;
@ -86,7 +86,7 @@ WHERE
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
o_orderkey < 0;
@ -101,7 +101,7 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
c_custkey < 0;
@ -119,7 +119,7 @@ WHERE
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
c_custkey < 0;
@ -135,7 +135,7 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;
DEBUG: join prunable for task partitionId 0 and 1
@ -183,7 +183,7 @@ DETAIL: Creating dependency on merge taskId 16
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;
DEBUG: join prunable for task partitionId 0 and 1
@ -225,7 +225,7 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey AND
l_orderkey < 0;
@ -246,7 +246,7 @@ WHERE
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey AND
l_orderkey < 0;
@ -260,7 +260,7 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey)
orders INNER JOIN customer_append ON (o_custkey = c_custkey)
WHERE
false;
QUERY PLAN
@ -277,7 +277,7 @@ WHERE
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey)
orders INNER JOIN customer_append ON (o_custkey = c_custkey)
WHERE
false;
o_orderkey
@ -288,7 +288,7 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey)
orders INNER JOIN customer_append ON (o_custkey = c_custkey)
WHERE
1=0 AND c_custkey < 0;
QUERY PLAN
@ -305,7 +305,7 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey AND false);
orders INNER JOIN customer_append ON (o_custkey = c_custkey AND false);
QUERY PLAN
----------------------------------
Custom Scan (Citus Task-Tracker)
@ -317,7 +317,7 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND false;
QUERY PLAN

View File

@ -23,7 +23,7 @@ SET citus.task_executor_type TO 'task-tracker';
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
DEBUG: assigned task 2 to node localhost:57637
@ -56,21 +56,20 @@ SET citus.large_table_shard_count TO 3;
SELECT
count(*)
FROM
orders, customer, lineitem
orders_reference, customer_append, lineitem
WHERE
o_custkey = c_custkey AND
o_orderkey = l_orderkey;
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 [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
DEBUG: join prunable for intervals [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
DEBUG: assigned task 3 to node localhost:57637
DEBUG: assigned task 5 to node localhost:57638
DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 6 to node localhost:57638
DEBUG: assigned task 2 to node localhost:57637
DEBUG: assigned task 3 to node localhost:57638
DEBUG: assigned task 1 to node localhost:57637
DEBUG: join prunable for intervals [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 3
DETAIL: Creating dependency on merge taskId 8
DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 2 to node localhost:57638
count
-------
@ -83,7 +82,7 @@ SET citus.large_table_shard_count TO 2;
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;
DEBUG: assigned task 2 to node localhost:57637
@ -119,10 +118,10 @@ DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 12
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 16
DEBUG: assigned task 3 to node localhost:57637
DEBUG: assigned task 6 to node localhost:57638
DEBUG: assigned task 9 to node localhost:57637
DEBUG: assigned task 12 to node localhost:57638
DEBUG: assigned task 3 to node localhost:57638
DEBUG: assigned task 6 to node localhost:57637
DEBUG: assigned task 9 to node localhost:57638
DEBUG: assigned task 12 to node localhost:57637
count
-------
125

View File

@ -30,7 +30,7 @@ DEBUG: CommitTransactionCommand
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
DEBUG: StartTransactionCommand
@ -68,22 +68,21 @@ DEBUG: CommitTransactionCommand
SELECT
count(*)
FROM
orders, customer, lineitem
orders_reference, customer_append, lineitem
WHERE
o_custkey = c_custkey AND
o_orderkey = l_orderkey;
DEBUG: StartTransactionCommand
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 [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
DEBUG: join prunable for intervals [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
DEBUG: assigned task 3 to node localhost:57637
DEBUG: assigned task 5 to node localhost:57638
DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 6 to node localhost:57638
DEBUG: assigned task 2 to node localhost:57637
DEBUG: assigned task 3 to node localhost:57638
DEBUG: assigned task 1 to node localhost:57637
DEBUG: join prunable for intervals [1,5986] and [8997,14947]
DEBUG: join prunable for intervals [8997,14947] and [1,5986]
DEBUG: pruning merge fetch taskId 1
DETAIL: Creating dependency on merge taskId 4
DEBUG: pruning merge fetch taskId 3
DETAIL: Creating dependency on merge taskId 8
DEBUG: assigned task 4 to node localhost:57637
DEBUG: assigned task 2 to node localhost:57638
DEBUG: CommitTransactionCommand
count
@ -100,7 +99,7 @@ DEBUG: CommitTransactionCommand
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;
DEBUG: StartTransactionCommand
@ -137,10 +136,10 @@ DEBUG: pruning merge fetch taskId 10
DETAIL: Creating dependency on merge taskId 12
DEBUG: pruning merge fetch taskId 11
DETAIL: Creating dependency on merge taskId 16
DEBUG: assigned task 3 to node localhost:57637
DEBUG: assigned task 6 to node localhost:57638
DEBUG: assigned task 9 to node localhost:57637
DEBUG: assigned task 12 to node localhost:57638
DEBUG: assigned task 3 to node localhost:57638
DEBUG: assigned task 6 to node localhost:57637
DEBUG: assigned task 9 to node localhost:57638
DEBUG: assigned task 12 to node localhost:57637
DEBUG: CommitTransactionCommand
count
-------

View File

@ -765,7 +765,7 @@ FROM
reference_table_test, colocated_table_test
WHERE
colocated_table_test.value_1 = reference_table_test.value_1;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ]
value_1
---------
1
@ -778,7 +778,7 @@ FROM
reference_table_test, colocated_table_test
WHERE
colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ]
value_2
---------
1
@ -791,7 +791,7 @@ FROM
colocated_table_test, reference_table_test
WHERE
reference_table_test.value_1 = colocated_table_test.value_1;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ]
value_2
---------
1
@ -804,7 +804,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ cartesian product "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ cartesian product "colocated_table_test_2" ]
ERROR: cannot perform distributed planning on this query
DETAIL: Cartesian products are currently unsupported
SELECT
@ -813,7 +813,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_1 = colocated_table_test_2.value_1 AND colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ local partition join "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ local partition join "colocated_table_test_2" ]
value_2
---------
1
@ -827,7 +827,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_2 = colocated_table_test_2.value_2 AND colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
value_2
---------
1
@ -840,7 +840,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_1 = reference_table_test.value_1 AND colocated_table_test_2.value_1 = reference_table_test.value_1;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
value_2
---------
1

View File

@ -156,19 +156,17 @@ INSERT INTO repartition_udt_other values (10, '(2,1)'::test_udt, 'foo');
INSERT INTO repartition_udt_other values (11, '(2,2)'::test_udt, 'foo');
INSERT INTO repartition_udt_other values (12, '(2,3)'::test_udt, 'foo');
SET client_min_messages = LOG;
-- This query was intended to test "Query that should result in a repartition
-- join on int column, and be empty." While removing broadcast logic, we
-- manually make the query router plannable.
SET citus.task_executor_type = 'task-tracker';
-- Query that should result in a repartition
-- join on int column, and be empty.
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.pk = repartition_udt_other.pk
WHERE repartition_udt.pk = 1;
ON repartition_udt.pk = repartition_udt_other.pk;
pk | udtcol | txtcol | pk | udtcol | txtcol
----+--------+--------+----+--------+--------
(0 rows)
-- Query that should result in a repartition join on UDT column.
SET citus.large_table_shard_count = 1;
SET citus.task_executor_type = 'task-tracker';
SET citus.log_multi_join_order = true;
EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.udtcol = repartition_udt_other.udtcol

View File

@ -7,12 +7,11 @@ SET client_min_messages = LOG;
SET citus.large_table_shard_count = 1;
SET citus.task_executor_type = 'task-tracker';
SET citus.log_multi_join_order = true;
-- This query was intended to test "Query that should result in a repartition
-- join on int column, and be empty." In order to remove broadcast logic, we
-- manually make the query router plannable.
-- Query that should result in a repartition
-- join on int column, and be empty
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.pk = repartition_udt_other.pk
WHERE repartition_udt.pk = 1;
ON repartition_udt.pk = repartition_udt_other.pk;
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
pk | udtcol | txtcol | pk | udtcol | txtcol
----+--------+--------+----+--------+--------
(0 rows)

View File

@ -7,12 +7,11 @@ SET client_min_messages = LOG;
SET citus.large_table_shard_count = 1;
SET citus.task_executor_type = 'task-tracker';
SET citus.log_multi_join_order = true;
-- This query was intended to test "Query that should result in a repartition
-- join on int column, and be empty." In order to remove broadcast logic, we
-- manually make the query router plannable.
-- Query that should result in a repartition
-- join on int column, and be empty.
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.pk = repartition_udt_other.pk
WHERE repartition_udt.pk = 1;
ON repartition_udt.pk = repartition_udt_other.pk;
LOG: join order: [ "repartition_udt" ][ dual partition join "repartition_udt_other" ]
pk | udtcol | txtcol | pk | udtcol | txtcol
----+--------+--------+----+--------+--------
(0 rows)

View File

@ -1013,7 +1013,7 @@ FROM
reference_table_test, colocated_table_test
WHERE
colocated_table_test.value_1 = reference_table_test.value_1;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ]
value_1
---------
1
@ -1026,7 +1026,7 @@ FROM
reference_table_test, colocated_table_test
WHERE
colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ]
value_2
---------
1
@ -1039,7 +1039,7 @@ FROM
colocated_table_test, reference_table_test
WHERE
reference_table_test.value_1 = colocated_table_test.value_1;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ]
value_2
---------
1
@ -1052,7 +1052,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ cartesian product "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ cartesian product "colocated_table_test_2" ]
ERROR: cannot perform distributed planning on this query
DETAIL: Cartesian products are currently unsupported
SELECT
@ -1061,7 +1061,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_1 = colocated_table_test_2.value_1 AND colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ local partition join "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ local partition join "colocated_table_test_2" ]
value_2
---------
1
@ -1075,7 +1075,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_2 = colocated_table_test_2.value_2 AND colocated_table_test.value_2 = reference_table_test.value_2;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
value_2
---------
1
@ -1088,7 +1088,7 @@ FROM
reference_table_test, colocated_table_test, colocated_table_test_2
WHERE
colocated_table_test.value_1 = reference_table_test.value_1 AND colocated_table_test_2.value_1 = reference_table_test.value_1;
LOG: join order: [ "colocated_table_test" ][ broadcast join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
LOG: join order: [ "colocated_table_test" ][ reference join "reference_table_test" ][ dual partition join "colocated_table_test_2" ]
value_2
---------
1

View File

@ -212,7 +212,7 @@ EXPLAIN (COSTS FALSE)
-> HashAggregate
Group Key: l_orderkey
Filter: (count(*) > 5)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
Filter: (l_orderkey < 200)
(15 rows)
@ -242,7 +242,7 @@ EXPLAIN (COSTS FALSE)
-> HashAggregate
Group Key: l_orderkey
Filter: (count(*) > 5)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
Filter: (l_orderkey < 200)
(17 rows)
@ -285,7 +285,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows)
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique
@ -314,7 +314,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(17 rows)
SET enable_hashagg TO on;
@ -363,7 +363,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(15 rows)
-- check the plan if the hash aggreate is disabled. Similar to the explain of
@ -394,7 +394,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(18 rows)
SET enable_hashagg TO on;
@ -444,7 +444,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(15 rows)
-- check the plan if the hash aggreate is disabled. This explain errors out due
@ -502,7 +502,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows)
-- check the plan if the hash aggreate is disabled. We expect to see sort+unique to
@ -531,7 +531,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(16 rows)
SET enable_hashagg TO on;
@ -579,7 +579,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(15 rows)
-- check the plan if the hash aggreate is disabled. This explain errors out due
@ -638,7 +638,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey, l_linenumber
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(15 rows)
-- check the plan if the hash aggreate is disabled. This explain errors out due
@ -702,7 +702,7 @@ EXPLAIN (COSTS FALSE)
Group 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
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_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(17 rows)
-- check the plan if the hash aggreate is disabled. We expect to see only one
@ -734,7 +734,7 @@ EXPLAIN (COSTS FALSE)
Group 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
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_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(19 rows)
@ -798,7 +798,7 @@ EXPLAIN (COSTS FALSE)
Group Key: l_orderkey
-> Sort
Sort Key: l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(16 rows)
-- check the plan if the hash aggreate is disabled. We expect to see sort + unique
@ -829,7 +829,7 @@ EXPLAIN (COSTS FALSE)
Group Key: l_orderkey
-> Sort
Sort Key: l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(19 rows)
@ -870,7 +870,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows)
-- check the plan if the hash aggreate is disabled
@ -898,7 +898,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_suppkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(17 rows)
@ -929,7 +929,7 @@ EXPLAIN (COSTS FALSE)
Group Key: l_orderkey
-> Sort
Sort Key: l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(17 rows)
-- check the plan if the hash aggreate is disabled.
@ -961,7 +961,7 @@ EXPLAIN (COSTS FALSE)
Group Key: l_orderkey
-> Sort
Sort Key: l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(20 rows)
@ -1010,7 +1010,7 @@ EXPLAIN (COSTS FALSE)
Node: host=localhost port=57637 dbname=regression
-> HashAggregate
Group Key: l_partkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(13 rows)
-- distinct on non-partition column and avg
@ -1083,7 +1083,7 @@ EXPLAIN (COSTS FALSE)
-> Unique
-> Sort
Sort Key: l_partkey, l_suppkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
Filter: ((l_orderkey < 100) AND (l_shipmode = 'AIR'::bpchar))
(14 rows)
@ -1125,7 +1125,7 @@ EXPLAIN (COSTS FALSE)
-> Unique
-> Sort
Sort Key: l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
Filter: (l_orderkey < 35)
(13 rows)
@ -1181,7 +1181,7 @@ EXPLAIN (COSTS FALSE)
-> Unique
-> Sort
Sort Key: l_partkey, l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(14 rows)
-- distinct on with joins
@ -1224,9 +1224,9 @@ EXPLAIN (COSTS FALSE)
Sort Key: orders_hash_part.o_custkey, lineitem_hash_part.l_orderkey
-> Hash Join
Hash Cond: (lineitem_hash_part.l_orderkey = orders_hash_part.o_orderkey)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Hash
-> Seq Scan on orders_hash_part_360042 orders_hash_part
-> Seq Scan on orders_hash_part_360045 orders_hash_part
Filter: (o_custkey < 15)
(17 rows)
@ -1251,9 +1251,9 @@ EXPLAIN (COSTS FALSE)
Sort Key: orders_hash_part.o_custkey
-> Hash Join
Hash Cond: (lineitem_hash_part.l_orderkey = orders_hash_part.o_orderkey)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Hash
-> Seq Scan on orders_hash_part_360042 orders_hash_part
-> Seq Scan on orders_hash_part_360045 orders_hash_part
Filter: (o_custkey < 15)
(17 rows)
@ -1322,9 +1322,9 @@ EXPLAIN (COSTS FALSE)
Sort Key: orders_hash_part.o_custkey, lineitem_hash_part.l_orderkey
-> Hash Join
Hash Cond: (lineitem_hash_part.l_orderkey = orders_hash_part.o_orderkey)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
-> Hash
-> Seq Scan on orders_hash_part_360042 orders_hash_part
-> Seq Scan on orders_hash_part_360045 orders_hash_part
Filter: (o_custkey < 20)
(17 rows)
@ -1412,7 +1412,7 @@ EXPLAIN (COSTS FALSE)
Sort Key: l_orderkey, l_partkey
-> HashAggregate
Group Key: l_orderkey, l_partkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(16 rows)
SELECT DISTINCT l_orderkey, cnt
@ -1465,7 +1465,7 @@ EXPLAIN (COSTS FALSE)
Group Key: lineitem_hash_part.l_orderkey, count(*)
-> HashAggregate
Group Key: lineitem_hash_part.l_orderkey
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(18 rows)
-- distinct on partition column
@ -1518,7 +1518,7 @@ EXPLAIN (COSTS FALSE)
Sort Key: q.l_orderkey, q.l_partkey
-> Subquery Scan on q
Filter: (q.r > 1)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(16 rows)
-- distinct on non-partition column
@ -1570,6 +1570,6 @@ EXPLAIN (COSTS FALSE)
Sort Key: q.l_partkey, q.l_orderkey
-> Subquery Scan on q
Filter: (q.r > 1)
-> Seq Scan on lineitem_hash_part_360038 lineitem_hash_part
-> Seq Scan on lineitem_hash_part_360041 lineitem_hash_part
(16 rows)

View File

@ -7,12 +7,12 @@ SET citus.next_shard_id TO 230000;
-- Initialize tables to join
CREATE TABLE multi_append_table_to_shard_right
CREATE TABLE multi_append_table_to_shard_right_reference
(
right_number INTEGER not null,
right_text TEXT not null
);
SELECT master_create_distributed_table('multi_append_table_to_shard_right', 'right_number', 'append');
SELECT create_reference_table('multi_append_table_to_shard_right_reference');
CREATE TABLE multi_append_table_to_shard_left
(
@ -21,13 +21,13 @@ CREATE TABLE multi_append_table_to_shard_left
);
SELECT master_create_distributed_table('multi_append_table_to_shard_left', 'left_number', 'append');
CREATE TABLE multi_append_table_to_shard_right_hash
CREATE TABLE multi_append_table_to_shard_right_reference_hash
(
right_number INTEGER not null,
right_text TEXT not null
);
SELECT master_create_distributed_table('multi_append_table_to_shard_right_hash', 'right_number', 'hash');
SELECT master_create_worker_shards('multi_append_table_to_shard_right_hash', 1, 1);
SELECT master_create_distributed_table('multi_append_table_to_shard_right_reference_hash', 'right_number', 'hash');
SELECT master_create_worker_shards('multi_append_table_to_shard_right_reference_hash', 1, 1);
-- Replicate 'left' table on both workers
SELECT set_config('citus.shard_replication_factor', '2', false);
@ -35,8 +35,7 @@ SELECT set_config('citus.shard_replication_factor', '2', false);
\copy multi_append_table_to_shard_left FROM '@abs_srcdir@/data/agg.data'
-- Place 'right' table on both workers
SELECT set_config('citus.shard_replication_factor', '2', false);
\copy multi_append_table_to_shard_right FROM '@abs_srcdir@/data/agg.data'
\copy multi_append_table_to_shard_right_reference FROM '@abs_srcdir@/data/agg.data'
-- Reset shard replication factor to ensure tasks will be assigned to both workers
SELECT set_config('citus.shard_replication_factor', '2', false);
@ -44,7 +43,7 @@ SELECT set_config('citus.shard_replication_factor', '2', false);
-- All 8 rows in left table match a row in right table
SELECT COUNT(*)
FROM multi_append_table_to_shard_left,
multi_append_table_to_shard_right
multi_append_table_to_shard_right_reference
WHERE left_number = right_number;
-- Now append more data to the 'right' table
@ -57,18 +56,17 @@ CREATE TABLE multi_append_table_to_shard_stage
COPY multi_append_table_to_shard_stage FROM '@abs_srcdir@/data/large_records.data' with delimiter '|';
-- Check that we error out if we try to append data to a hash partitioned table.
SELECT master_create_empty_shard('multi_append_table_to_shard_right_hash');
SELECT master_create_empty_shard('multi_append_table_to_shard_right_reference_hash');
SELECT master_append_table_to_shard(shardid, 'multi_append_table_to_shard_stage', 'localhost', 57636)
FROM
pg_dist_shard
WHERE 'multi_append_table_to_shard_right_hash'::regclass::oid = logicalrelid;
WHERE 'multi_append_table_to_shard_right_reference_hash'::regclass::oid = logicalrelid;
-- Clean up after test
SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_right');
SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_left');
DROP TABLE multi_append_table_to_shard_stage;
DROP TABLE multi_append_table_to_shard_right;
DROP TABLE multi_append_table_to_shard_right_reference;
DROP TABLE multi_append_table_to_shard_left;
-- Check partitioning by date

View File

@ -14,9 +14,14 @@ SET citus.next_shard_id TO 290000;
\copy orders FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|'
\copy orders FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|'
\copy orders_reference FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|'
\copy orders_reference FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|'
\copy customer FROM '@abs_srcdir@/data/customer.1.data' with delimiter '|'
\copy customer_append FROM '@abs_srcdir@/data/customer.1.data' with delimiter '|'
\copy nation FROM '@abs_srcdir@/data/nation.data' with delimiter '|'
\copy part FROM '@abs_srcdir@/data/part.data' with delimiter '|'
\copy part_append FROM '@abs_srcdir@/data/part.data' with delimiter '|'
\copy supplier FROM '@abs_srcdir@/data/supplier.data' with delimiter '|'
\copy supplier_single_shard FROM '@abs_srcdir@/data/supplier.data' with delimiter '|'
\copy lineitem_hash_part FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|'

View File

@ -14,6 +14,10 @@ SET citus.next_shard_id TO 280000;
\copy customer FROM '@abs_srcdir@/data/customer.3.data' with delimiter '|'
\copy part FROM '@abs_srcdir@/data/part.more.data' with delimiter '|'
\copy customer_append FROM '@abs_srcdir@/data/customer.2.data' with delimiter '|'
\copy customer_append FROM '@abs_srcdir@/data/customer.3.data' with delimiter '|'
\copy part_append FROM '@abs_srcdir@/data/part.more.data' with delimiter '|'
-- Exchange partition files in binary format in remaining tests
SELECT success FROM run_command_on_workers('ALTER SYSTEM SET citus.binary_worker_copy_format TO on');
SELECT success FROM run_command_on_workers('SELECT pg_reload_conf()');

View File

@ -32,6 +32,19 @@ CREATE TABLE multi_outer_join_right
);
SELECT master_create_distributed_table('multi_outer_join_right', 'r_custkey', 'append');
CREATE TABLE multi_outer_join_right_reference
(
r_custkey integer not null,
r_name varchar(25) not null,
r_address varchar(40) not null,
r_nationkey integer not null,
r_phone char(15) not null,
r_acctbal decimal(15,2) not null,
r_mktsegment char(10) not null,
r_comment varchar(117) not null
);
SELECT create_reference_table('multi_outer_join_right_reference');
CREATE TABLE multi_outer_join_third
(
t_custkey integer not null,
@ -45,6 +58,20 @@ CREATE TABLE multi_outer_join_third
);
SELECT master_create_distributed_table('multi_outer_join_third', 't_custkey', 'append');
CREATE TABLE multi_outer_join_third_reference
(
t_custkey integer not null,
t_name varchar(25) not null,
t_address varchar(40) not null,
t_nationkey integer not null,
t_phone char(15) not null,
t_acctbal decimal(15,2) not null,
t_mktsegment char(10) not null,
t_comment varchar(117) not null
);
SELECT create_reference_table('multi_outer_join_third_reference');
-- Make sure we do not crash if both tables have no shards
SELECT
min(l_custkey), max(l_custkey)
@ -57,6 +84,7 @@ FROM
-- Right table is a small table
\copy multi_outer_join_right FROM '@abs_srcdir@/data/customer-1-15.data' with delimiter '|'
\copy multi_outer_join_right_reference FROM '@abs_srcdir@/data/customer-1-15.data' with delimiter '|'
-- Make sure we do not crash if one table has no shards
SELECT
@ -67,29 +95,30 @@ FROM
SELECT
min(t_custkey), max(t_custkey)
FROM
multi_outer_join_third a LEFT JOIN multi_outer_join_right b ON (r_custkey = t_custkey);
multi_outer_join_third a LEFT JOIN multi_outer_join_right_reference b ON (r_custkey = t_custkey);
-- Third table is a single shard table with all data
\copy multi_outer_join_third FROM '@abs_srcdir@/data/customer-1-30.data' with delimiter '|'
\copy multi_outer_join_third_reference FROM '@abs_srcdir@/data/customer-1-30.data' with delimiter '|'
-- Regular outer join should return results for all rows
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
-- Since this is a broadcast join, we should be able to join on any key
SELECT
count(*)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_nationkey = r_nationkey);
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_nationkey = r_nationkey);
-- Anti-join should return customers for which there is no row in the right table
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
@ -98,7 +127,7 @@ WHERE
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 5;
@ -108,7 +137,7 @@ WHERE
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 5 or r_custkey > 15;
@ -117,21 +146,21 @@ WHERE
SELECT
count(l_custkey), count(r_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 5);
-- Apply a filter before the join (no matches right)
SELECT
count(l_custkey), count(r_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = -1 /* nonexistant */);
-- Apply a filter before the join (no matches left)
SELECT
count(l_custkey), count(r_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */);
-- Right join should be disallowed in this case
@ -145,7 +174,7 @@ FROM
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
-- Turn the right table into a large table
@ -260,7 +289,7 @@ SELECT
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third t1 ON (r1.r_custkey = t1.t_custkey);
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
-- Right join with single shard right most table should error out
SELECT
@ -268,13 +297,13 @@ SELECT
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_third t1 ON (r1.r_custkey = t1.t_custkey);
RIGHT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
-- Right join with single shard left most table should work
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
@ -282,7 +311,7 @@ FROM
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey)
WHERE
@ -292,7 +321,7 @@ WHERE
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
@ -343,21 +372,21 @@ SELECT
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third t1 ON (r1.r_custkey = t1.t_custkey);
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
-- inner (broadcast) join + 2 shards left (local) join should work
SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey)
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
-- inner (local) join + 2 shards left (dual partition) join should error out
SELECT
t_custkey, l_custkey, r_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
INNER JOIN multi_outer_join_left l1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
@ -366,7 +395,7 @@ SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey)
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
-- inner (broadcast) join + 2 shards left (local) + anti join should work
@ -374,7 +403,7 @@ SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey)
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
WHERE
r_custkey is NULL;
@ -386,7 +415,7 @@ FROM
(multi_outer_join_right r1
LEFT OUTER JOIN multi_outer_join_left l1 ON (l1.l_custkey = r1.r_custkey)) AS
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third t1 ON (test.c_custkey = t1.t_custkey);
INNER JOIN multi_outer_join_third_reference t1 ON (test.c_custkey = t1.t_custkey);
-- flattened out subqueries with outer joins are not supported
SELECT

View File

@ -3,14 +3,14 @@
--
SET citus.next_shard_id TO 230000;
-- Initialize tables to join
CREATE TABLE multi_append_table_to_shard_right
CREATE TABLE multi_append_table_to_shard_right_reference
(
right_number INTEGER not null,
right_text TEXT not null
);
SELECT master_create_distributed_table('multi_append_table_to_shard_right', 'right_number', 'append');
master_create_distributed_table
---------------------------------
SELECT create_reference_table('multi_append_table_to_shard_right_reference');
create_reference_table
------------------------
(1 row)
@ -25,18 +25,18 @@ SELECT master_create_distributed_table('multi_append_table_to_shard_left', 'left
(1 row)
CREATE TABLE multi_append_table_to_shard_right_hash
CREATE TABLE multi_append_table_to_shard_right_reference_hash
(
right_number INTEGER not null,
right_text TEXT not null
);
SELECT master_create_distributed_table('multi_append_table_to_shard_right_hash', 'right_number', 'hash');
SELECT master_create_distributed_table('multi_append_table_to_shard_right_reference_hash', 'right_number', 'hash');
master_create_distributed_table
---------------------------------
(1 row)
SELECT master_create_worker_shards('multi_append_table_to_shard_right_hash', 1, 1);
SELECT master_create_worker_shards('multi_append_table_to_shard_right_reference_hash', 1, 1);
master_create_worker_shards
-----------------------------
@ -52,13 +52,7 @@ SELECT set_config('citus.shard_replication_factor', '2', false);
\copy multi_append_table_to_shard_left FROM '@abs_srcdir@/data/agg.data'
\copy multi_append_table_to_shard_left FROM '@abs_srcdir@/data/agg.data'
-- Place 'right' table on both workers
SELECT set_config('citus.shard_replication_factor', '2', false);
set_config
------------
2
(1 row)
\copy multi_append_table_to_shard_right FROM '@abs_srcdir@/data/agg.data'
\copy multi_append_table_to_shard_right_reference FROM '@abs_srcdir@/data/agg.data'
-- Reset shard replication factor to ensure tasks will be assigned to both workers
SELECT set_config('citus.shard_replication_factor', '2', false);
set_config
@ -69,7 +63,7 @@ SELECT set_config('citus.shard_replication_factor', '2', false);
-- All 8 rows in left table match a row in right table
SELECT COUNT(*)
FROM multi_append_table_to_shard_left,
multi_append_table_to_shard_right
multi_append_table_to_shard_right_reference
WHERE left_number = right_number;
count
-------
@ -84,22 +78,16 @@ CREATE TABLE multi_append_table_to_shard_stage
);
COPY multi_append_table_to_shard_stage FROM '@abs_srcdir@/data/large_records.data' with delimiter '|';
-- Check that we error out if we try to append data to a hash partitioned table.
SELECT master_create_empty_shard('multi_append_table_to_shard_right_hash');
ERROR: relation "multi_append_table_to_shard_right_hash" is a hash partitioned table
SELECT master_create_empty_shard('multi_append_table_to_shard_right_reference_hash');
ERROR: relation "multi_append_table_to_shard_right_reference_hash" is a hash partitioned table
DETAIL: We currently don't support creating shards on hash-partitioned tables
SELECT master_append_table_to_shard(shardid, 'multi_append_table_to_shard_stage', 'localhost', 57636)
FROM
pg_dist_shard
WHERE 'multi_append_table_to_shard_right_hash'::regclass::oid = logicalrelid;
ERROR: cannot append to shardId 230000
WHERE 'multi_append_table_to_shard_right_reference_hash'::regclass::oid = logicalrelid;
ERROR: cannot append to shardId 230001
DETAIL: We currently don't support appending to shards in hash-partitioned or reference tables
-- Clean up after test
SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_right');
master_apply_delete_command
-----------------------------
1
(1 row)
SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_left');
master_apply_delete_command
-----------------------------
@ -107,7 +95,7 @@ SELECT master_apply_delete_command('DELETE FROM multi_append_table_to_shard_left
(1 row)
DROP TABLE multi_append_table_to_shard_stage;
DROP TABLE multi_append_table_to_shard_right;
DROP TABLE multi_append_table_to_shard_right_reference;
DROP TABLE multi_append_table_to_shard_left;
-- Check partitioning by date
CREATE TABLE multi_append_table_to_shard_date

View File

@ -767,8 +767,13 @@ SELECT shardid, nodename, nodeport
-- add the node back
SELECT 1 FROM master_activate_node('localhost', :worker_1_port);
NOTICE: Replicating reference table "orders_reference" to the node localhost:57637
NOTICE: Replicating reference table "customer" to the node localhost:57637
NOTICE: Replicating reference table "nation" to the node localhost:57637
NOTICE: Replicating reference table "part" to the node localhost:57637
NOTICE: Replicating reference table "supplier" to the node localhost:57637
NOTICE: Replicating reference table "multi_outer_join_right_reference" to the node localhost:57637
NOTICE: Replicating reference table "multi_outer_join_third_reference" to the node localhost:57637
?column?
----------
1

View File

@ -10,9 +10,13 @@ SET citus.next_shard_id TO 290000;
\copy lineitem FROM '@abs_srcdir@/data/lineitem.2.data' with delimiter '|'
\copy orders FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|'
\copy orders FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|'
\copy orders_reference FROM '@abs_srcdir@/data/orders.1.data' with delimiter '|'
\copy orders_reference FROM '@abs_srcdir@/data/orders.2.data' with delimiter '|'
\copy customer FROM '@abs_srcdir@/data/customer.1.data' with delimiter '|'
\copy customer_append FROM '@abs_srcdir@/data/customer.1.data' with delimiter '|'
\copy nation FROM '@abs_srcdir@/data/nation.data' with delimiter '|'
\copy part FROM '@abs_srcdir@/data/part.data' with delimiter '|'
\copy part_append FROM '@abs_srcdir@/data/part.data' with delimiter '|'
\copy supplier FROM '@abs_srcdir@/data/supplier.data' with delimiter '|'
\copy supplier_single_shard FROM '@abs_srcdir@/data/supplier.data' with delimiter '|'
\copy lineitem_hash_part FROM '@abs_srcdir@/data/lineitem.1.data' with delimiter '|'

View File

@ -8,6 +8,9 @@ SET citus.next_shard_id TO 280000;
\copy customer FROM '@abs_srcdir@/data/customer.2.data' with delimiter '|'
\copy customer FROM '@abs_srcdir@/data/customer.3.data' with delimiter '|'
\copy part FROM '@abs_srcdir@/data/part.more.data' with delimiter '|'
\copy customer_append FROM '@abs_srcdir@/data/customer.2.data' with delimiter '|'
\copy customer_append FROM '@abs_srcdir@/data/customer.3.data' with delimiter '|'
\copy part_append FROM '@abs_srcdir@/data/part.more.data' with delimiter '|'
-- Exchange partition files in binary format in remaining tests
SELECT success FROM run_command_on_workers('ALTER SYSTEM SET citus.binary_worker_copy_format TO on');
success

View File

@ -36,6 +36,23 @@ SELECT master_create_distributed_table('multi_outer_join_right', 'r_custkey', 'a
(1 row)
CREATE TABLE multi_outer_join_right_reference
(
r_custkey integer not null,
r_name varchar(25) not null,
r_address varchar(40) not null,
r_nationkey integer not null,
r_phone char(15) not null,
r_acctbal decimal(15,2) not null,
r_mktsegment char(10) not null,
r_comment varchar(117) not null
);
SELECT create_reference_table('multi_outer_join_right_reference');
create_reference_table
------------------------
(1 row)
CREATE TABLE multi_outer_join_third
(
t_custkey integer not null,
@ -53,18 +70,40 @@ SELECT master_create_distributed_table('multi_outer_join_third', 't_custkey', 'a
(1 row)
CREATE TABLE multi_outer_join_third_reference
(
t_custkey integer not null,
t_name varchar(25) not null,
t_address varchar(40) not null,
t_nationkey integer not null,
t_phone char(15) not null,
t_acctbal decimal(15,2) not null,
t_mktsegment char(10) not null,
t_comment varchar(117) not null
);
SELECT create_reference_table('multi_outer_join_third_reference');
create_reference_table
------------------------
(1 row)
-- Make sure we do not crash if both tables have no shards
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_third b ON (l_custkey = t_custkey);
ERROR: cannot perform distributed planning on this query
DETAIL: Shards of relations in outer join queries must have 1-to-1 shard partitioning
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_third" ]
min | max
-----+-----
|
(1 row)
-- Left table is a large table
\copy multi_outer_join_left FROM '@abs_srcdir@/data/customer-1-10.data' with delimiter '|'
\copy multi_outer_join_left FROM '@abs_srcdir@/data/customer-11-20.data' with delimiter '|'
-- Right table is a small table
\copy multi_outer_join_right FROM '@abs_srcdir@/data/customer-1-15.data' with delimiter '|'
\copy multi_outer_join_right_reference FROM '@abs_srcdir@/data/customer-1-15.data' with delimiter '|'
-- Make sure we do not crash if one table has no shards
SELECT
min(l_custkey), max(l_custkey)
@ -75,8 +114,8 @@ DETAIL: Outer joins requiring repartitioning are not supported.
SELECT
min(t_custkey), max(t_custkey)
FROM
multi_outer_join_third a LEFT JOIN multi_outer_join_right b ON (r_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_third" ][ broadcast join "multi_outer_join_right" ]
multi_outer_join_third a LEFT JOIN multi_outer_join_right_reference b ON (r_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_third" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
|
@ -84,12 +123,13 @@ LOG: join order: [ "multi_outer_join_third" ][ broadcast join "multi_outer_join
-- Third table is a single shard table with all data
\copy multi_outer_join_third FROM '@abs_srcdir@/data/customer-1-30.data' with delimiter '|'
\copy multi_outer_join_third_reference FROM '@abs_srcdir@/data/customer-1-30.data' with delimiter '|'
-- Regular outer join should return results for all rows
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -99,8 +139,8 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
count(*)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_nationkey = r_nationkey);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_nationkey = r_nationkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count
-------
28
@ -110,10 +150,10 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
16 | 20
@ -123,10 +163,10 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 5;
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 20
@ -137,10 +177,10 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b ON (l_custkey = r_custkey)
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 5 or r_custkey > 15;
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 5
@ -150,9 +190,9 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
count(l_custkey), count(r_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 5);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 1
@ -162,9 +202,9 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
count(l_custkey), count(r_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -174,9 +214,9 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
count(l_custkey), count(r_custkey)
FROM
multi_outer_join_left a LEFT JOIN multi_outer_join_right b
multi_outer_join_left a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -193,8 +233,8 @@ DETAIL: Outer joins requiring repartitioning are not supported.
SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_right" ]
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -346,8 +386,8 @@ SELECT
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third t1 ON (r1.r_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ][ broadcast join "multi_outer_join_third" ]
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -375,16 +415,16 @@ SELECT
FROM
multi_outer_join_left l1
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_third t1 ON (r1.r_custkey = t1.t_custkey);
RIGHT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
ERROR: could not run distributed query with complex join orders
-- Right join with single shard left most table should work
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
LOG: join order: [ "multi_outer_join_right" ][ broadcast join "multi_outer_join_third" ][ local partition join "multi_outer_join_left" ]
LOG: join order: [ "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -410,12 +450,12 @@ LOG: join order: [ "multi_outer_join_right" ][ broadcast join "multi_outer_join
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey)
WHERE
l_custkey is NULL;
LOG: join order: [ "multi_outer_join_right" ][ broadcast join "multi_outer_join_third" ][ local partition join "multi_outer_join_left" ]
LOG: join order: [ "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
13 | 13 |
@ -427,7 +467,7 @@ LOG: join order: [ "multi_outer_join_right" ][ broadcast join "multi_outer_join
SELECT
t_custkey, r_custkey, l_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
RIGHT JOIN multi_outer_join_right r1 ON (t1.t_custkey = r1.r_custkey)
RIGHT JOIN multi_outer_join_left l1 ON (r1.r_custkey = l1.l_custkey);
ERROR: could not run distributed query with complex join orders
@ -527,8 +567,8 @@ SELECT
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third t1 ON (r1.r_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ][ broadcast join "multi_outer_join_third" ]
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_left" ][ local partition join "multi_outer_join_right" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -552,9 +592,9 @@ SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey)
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_third" ][ local partition join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -580,7 +620,7 @@ LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_
SELECT
t_custkey, l_custkey, r_custkey
FROM
multi_outer_join_third t1
multi_outer_join_third_reference t1
INNER JOIN multi_outer_join_left l1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
ERROR: cannot run outer join query if join is not on the partition column
@ -590,9 +630,9 @@ SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey)
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey);
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_third" ][ local partition join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -619,11 +659,11 @@ SELECT
l_custkey, t_custkey, r_custkey
FROM
multi_outer_join_left l1
INNER JOIN multi_outer_join_third t1 ON (l1.l_custkey = t1.t_custkey)
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right r1 ON (l1.l_custkey = r1.r_custkey)
WHERE
r_custkey is NULL;
LOG: join order: [ "multi_outer_join_left" ][ broadcast join "multi_outer_join_third" ][ local partition join "multi_outer_join_right" ]
LOG: join order: [ "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
23 | 23 |
@ -638,8 +678,8 @@ FROM
(multi_outer_join_right r1
LEFT OUTER JOIN multi_outer_join_left l1 ON (l1.l_custkey = r1.r_custkey)) AS
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third t1 ON (test.c_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_right" ][ local partition join "multi_outer_join_left" ][ broadcast join "multi_outer_join_third" ]
INNER JOIN multi_outer_join_third_reference t1 ON (test.c_custkey = t1.t_custkey);
LOG: join order: [ "multi_outer_join_right" ][ local partition join "multi_outer_join_left" ][ reference join "multi_outer_join_third_reference" ]
t_custkey
-----------
11

View File

@ -75,7 +75,7 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_third_reference b ON (l_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_third_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ]
min | max
-----+-----
|
@ -92,7 +92,7 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_third_reference b ON (l_custkey = t_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_third_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ]
min | max
-----+-----
1 | 20
@ -115,7 +115,7 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -126,7 +126,7 @@ SELECT
count(*)
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_nationkey = r_nationkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count
-------
28
@ -139,7 +139,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
16 | 20
@ -152,7 +152,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 5;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 20
@ -166,7 +166,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 5 or r_custkey > 15;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
5 | 5
@ -178,7 +178,7 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 5);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 1
@ -190,7 +190,7 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -202,7 +202,7 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND l_custkey = -1 /* nonexistant */);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
20 | 0
@ -220,7 +220,7 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left_hash b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 20
@ -285,7 +285,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 10
@ -298,7 +298,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey IS NULL OR r_custkey = 15;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 15
@ -312,7 +312,7 @@ FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b ON (l_custkey = r_custkey)
WHERE
r_custkey = 21 or r_custkey < 10;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
21 | 21
@ -324,7 +324,7 @@ SELECT
FROM
multi_outer_join_left_hash a LEFT JOIN multi_outer_join_right_reference b
ON (l_custkey = r_custkey AND r_custkey = 21);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
count | count
-------+-------
25 | 1
@ -342,7 +342,7 @@ SELECT
min(l_custkey), max(l_custkey)
FROM
multi_outer_join_right_reference a RIGHT JOIN multi_outer_join_left_hash b ON (l_custkey = r_custkey);
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ]
min | max
-----+-----
1 | 30
@ -378,7 +378,7 @@ FROM
LEFT JOIN multi_outer_join_right_reference r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey)
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_right_reference" ][ broadcast join "multi_outer_join_third_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_right_reference" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
1 | |
@ -425,7 +425,7 @@ FROM
RIGHT JOIN multi_outer_join_right_hash r1 ON (t1.t_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_left_hash l1 ON (r1.r_custkey = l1.l_custkey)
ORDER BY 1,2,3;
LOG: join order: [ "multi_outer_join_right_hash" ][ broadcast join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left_hash" ]
LOG: join order: [ "multi_outer_join_right_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left_hash" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -460,7 +460,7 @@ FROM
WHERE
l_custkey is NULL
ORDER BY 1;
LOG: join order: [ "multi_outer_join_right_hash" ][ broadcast join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left_hash" ]
LOG: join order: [ "multi_outer_join_right_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_left_hash" ]
t_custkey | r_custkey | l_custkey
-----------+-----------+-----------
16 | 16 |
@ -609,7 +609,7 @@ FROM
INNER JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
LEFT JOIN multi_outer_join_third_reference t1 ON (r1.r_custkey = t1.t_custkey)
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ][ broadcast join "multi_outer_join_third_reference" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ local partition join "multi_outer_join_right_hash" ][ reference join "multi_outer_join_third_reference" ]
l_custkey | r_custkey | t_custkey
-----------+-----------+-----------
11 | 11 | 11
@ -637,7 +637,7 @@ FROM
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
@ -684,7 +684,7 @@ FROM
INNER JOIN multi_outer_join_third_reference t1 ON (l1.l_custkey = t1.t_custkey)
LEFT JOIN multi_outer_join_right_hash r1 ON (l1.l_custkey = r1.r_custkey)
ORDER BY 1,2,3;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
@ -724,7 +724,7 @@ FROM
WHERE
r_custkey is NULL
ORDER BY 1;
LOG: join order: [ "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
LOG: join order: [ "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ][ local partition join "multi_outer_join_right_hash" ]
l_custkey | t_custkey | r_custkey
-----------+-----------+-----------
1 | 1 |
@ -748,7 +748,7 @@ FROM
test(c_custkey, c_nationkey)
INNER JOIN multi_outer_join_third_reference t1 ON (test.c_custkey = t1.t_custkey)
ORDER BY 1;
LOG: join order: [ "multi_outer_join_right_hash" ][ local partition join "multi_outer_join_left_hash" ][ broadcast join "multi_outer_join_third_reference" ]
LOG: join order: [ "multi_outer_join_right_hash" ][ local partition join "multi_outer_join_left_hash" ][ reference join "multi_outer_join_third_reference" ]
t_custkey
-----------
11

View File

@ -6,7 +6,9 @@ ALTER SEQUENCE pg_catalog.pg_dist_shardid_seq RESTART 360000;
ALTER SEQUENCE pg_catalog.pg_dist_colocationid_seq RESTART 100000;
-- Create new table definitions for use in testing in distributed planning and
-- execution functionality. Also create indexes to boost performance.
-- execution functionality. Also create indexes to boost performance. Since we
-- need to cover both reference join and partitioned join, we have created
-- reference and append distributed version of orders, customer and part tables.
CREATE TABLE lineitem (
l_orderkey bigint not null,
@ -43,6 +45,20 @@ CREATE TABLE orders (
PRIMARY KEY(o_orderkey) );
SELECT master_create_distributed_table('orders', 'o_orderkey', 'append');
CREATE TABLE orders_reference (
o_orderkey bigint not null,
o_custkey integer not null,
o_orderstatus char(1) not null,
o_totalprice decimal(15,2) not null,
o_orderdate date not null,
o_orderpriority char(15) not null,
o_clerk char(15) not null,
o_shippriority integer not null,
o_comment varchar(79) not null,
PRIMARY KEY(o_orderkey) );
SELECT create_reference_table('orders_reference');
CREATE TABLE customer (
c_custkey integer not null,
c_name varchar(25) not null,
@ -52,7 +68,18 @@ CREATE TABLE customer (
c_acctbal decimal(15,2) not null,
c_mktsegment char(10) not null,
c_comment varchar(117) not null);
SELECT master_create_distributed_table('customer', 'c_custkey', 'append');
SELECT create_reference_table('customer');
CREATE TABLE customer_append (
c_custkey integer not null,
c_name varchar(25) not null,
c_address varchar(40) not null,
c_nationkey integer not null,
c_phone char(15) not null,
c_acctbal decimal(15,2) not null,
c_mktsegment char(10) not null,
c_comment varchar(117) not null);
SELECT master_create_distributed_table('customer_append', 'c_custkey', 'append');
CREATE TABLE nation (
n_nationkey integer not null,
@ -72,7 +99,19 @@ CREATE TABLE part (
p_container char(10) not null,
p_retailprice decimal(15,2) not null,
p_comment varchar(23) not null);
SELECT master_create_distributed_table('part', 'p_partkey', 'append');
SELECT create_reference_table('part');
CREATE TABLE part_append (
p_partkey integer not null,
p_name varchar(55) not null,
p_mfgr char(25) not null,
p_brand char(10) not null,
p_type varchar(25) not null,
p_size integer not null,
p_container char(10) not null,
p_retailprice decimal(15,2) not null,
p_comment varchar(23) not null);
SELECT master_create_distributed_table('part_append', 'p_partkey', 'append');
CREATE TABLE supplier
(
@ -380,8 +419,8 @@ COMMIT;
-- Table should exist on the worker node
\c - - - :worker_1_port
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360066'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt2_360070'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360069'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt2_360073'::regclass;
\c - - - :master_port
DROP TABLE tt1;
@ -397,7 +436,7 @@ ROLLBACK;
-- Table exists on the worker node.
\c - - - :worker_1_port
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.append_tt1_360074'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.append_tt1_360077'::regclass;
\c - - - :master_port
-- There should be no table on the worker node
@ -417,7 +456,7 @@ COMMIT;
-- Placements should be created on the worker
\c - - - :worker_1_port
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360075'::regclass;
SELECT "Column", "Type", "Modifiers" FROM table_desc WHERE relid = 'public.tt1_360078'::regclass;
\c - - - :master_port
DROP TABLE tt1;

View File

@ -404,35 +404,35 @@ SET citus.large_table_shard_count TO 1;
EXPLAIN (COSTS FALSE)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
EXPLAIN (COSTS FALSE, FORMAT JSON)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
SELECT true AS valid FROM explain_json($$
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey$$);
EXPLAIN (COSTS FALSE, FORMAT XML)
SELECT count(*)
FROM lineitem, orders, customer, supplier_single_shard
FROM lineitem, orders, customer_append, supplier_single_shard
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey;
SELECT true AS valid FROM explain_xml($$
SELECT count(*)
FROM lineitem, orders, customer, supplier
FROM lineitem, orders, customer_append, supplier
WHERE l_orderkey = o_orderkey
AND o_custkey = c_custkey
AND l_suppkey = s_suppkey$$);

View File

@ -108,7 +108,7 @@ SET citus.large_table_shard_count TO 1;
-- 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.
EXPLAIN SELECT count(*) FROM orders, lineitem, customer
EXPLAIN SELECT count(*) FROM orders, lineitem, customer_append
WHERE o_custkey = l_partkey AND o_custkey = c_nationkey;
-- Validate that we don't chose a single-partition join method with a
@ -118,7 +118,7 @@ EXPLAIN SELECT count(*) FROM orders, customer_hash
-- Validate that we can re-partition a hash partitioned table to join with a
-- range partitioned one.
EXPLAIN SELECT count(*) FROM orders_hash, customer
EXPLAIN SELECT count(*) FROM orders_hash, customer_append
WHERE c_custkey = o_custkey;
COMMIT;

View File

@ -40,7 +40,7 @@ EXPLAIN SELECT
o_orderdate,
o_shippriority
FROM
customer,
customer_append,
orders,
lineitem
WHERE
@ -69,7 +69,7 @@ EXPLAIN SELECT
c_phone,
c_comment
FROM
customer,
customer_append,
orders,
lineitem,
nation
@ -97,7 +97,7 @@ EXPLAIN SELECT
sum(l_extendedprice* (1 - l_discount)) as revenue
FROM
lineitem,
part
part_append
WHERE
(
p_partkey = l_partkey
@ -128,7 +128,7 @@ WHERE
EXPLAIN SELECT
l_partkey, count(*)
FROM
lineitem, part, orders, customer
lineitem, part_append, orders, customer_append
WHERE
l_orderkey = o_orderkey AND
l_partkey = p_partkey AND

View File

@ -32,7 +32,7 @@ SET citus.task_executor_type TO 'task-tracker';
SELECT
l_partkey, o_orderkey, count(*)
FROM
lineitem, part, orders, customer
lineitem, part_append, orders, customer_append
WHERE
l_orderkey = o_orderkey AND
l_partkey = p_partkey AND

View File

@ -18,13 +18,13 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
@ -34,14 +34,14 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
o_orderkey < 0;
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
o_orderkey < 0;
@ -52,14 +52,14 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
c_custkey < 0;
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND
c_custkey < 0;
@ -71,13 +71,13 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;
@ -87,14 +87,14 @@ EXPLAIN (COSTS OFF)
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey AND
l_orderkey < 0;
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey AND
l_orderkey < 0;
@ -104,14 +104,14 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey)
orders INNER JOIN customer_append ON (o_custkey = c_custkey)
WHERE
false;
-- execute once, to verify that's handled
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey)
orders INNER JOIN customer_append ON (o_custkey = c_custkey)
WHERE
false;
@ -119,7 +119,7 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey)
orders INNER JOIN customer_append ON (o_custkey = c_custkey)
WHERE
1=0 AND c_custkey < 0;
@ -127,12 +127,12 @@ EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders INNER JOIN customer ON (o_custkey = c_custkey AND false);
orders INNER JOIN customer_append ON (o_custkey = c_custkey AND false);
EXPLAIN (COSTS OFF)
SELECT
o_orderkey
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey AND false;

View File

@ -25,7 +25,7 @@ SET citus.task_executor_type TO 'task-tracker';
SELECT
count(*)
FROM
orders, customer
orders, customer_append
WHERE
o_custkey = c_custkey;
@ -39,20 +39,19 @@ SET citus.large_table_shard_count TO 3;
SELECT
count(*)
FROM
orders, customer, lineitem
orders_reference, customer_append, lineitem
WHERE
o_custkey = c_custkey AND
o_orderkey = l_orderkey;
SET citus.large_table_shard_count TO 2;
-- Dual hash repartition join which tests the separate hash repartition join
-- task assignment algorithm.
SELECT
count(*)
FROM
lineitem, customer
lineitem, customer_append
WHERE
l_partkey = c_nationkey;

View File

@ -191,17 +191,15 @@ INSERT INTO repartition_udt_other values (11, '(2,2)'::test_udt, 'foo');
INSERT INTO repartition_udt_other values (12, '(2,3)'::test_udt, 'foo');
SET client_min_messages = LOG;
SET citus.task_executor_type = 'task-tracker';
-- This query was intended to test "Query that should result in a repartition
-- join on int column, and be empty." While removing broadcast logic, we
-- manually make the query router plannable.
-- Query that should result in a repartition
-- join on int column, and be empty.
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.pk = repartition_udt_other.pk
WHERE repartition_udt.pk = 1;
ON repartition_udt.pk = repartition_udt_other.pk;
-- Query that should result in a repartition join on UDT column.
SET citus.large_table_shard_count = 1;
SET citus.task_executor_type = 'task-tracker';
SET citus.log_multi_join_order = true;
EXPLAIN SELECT * FROM repartition_udt JOIN repartition_udt_other

View File

@ -9,12 +9,10 @@ SET citus.large_table_shard_count = 1;
SET citus.task_executor_type = 'task-tracker';
SET citus.log_multi_join_order = true;
-- This query was intended to test "Query that should result in a repartition
-- join on int column, and be empty." In order to remove broadcast logic, we
-- manually make the query router plannable.
-- Query that should result in a repartition
-- join on int column, and be empty
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.pk = repartition_udt_other.pk
WHERE repartition_udt.pk = 1;
ON repartition_udt.pk = repartition_udt_other.pk;
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.udtcol = repartition_udt_other.udtcol

View File

@ -9,12 +9,10 @@ SET citus.large_table_shard_count = 1;
SET citus.task_executor_type = 'task-tracker';
SET citus.log_multi_join_order = true;
-- This query was intended to test "Query that should result in a repartition
-- join on int column, and be empty." In order to remove broadcast logic, we
-- manually make the query router plannable.
-- Query that should result in a repartition
-- join on int column, and be empty.
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.pk = repartition_udt_other.pk
WHERE repartition_udt.pk = 1;
ON repartition_udt.pk = repartition_udt_other.pk;
SELECT * FROM repartition_udt JOIN repartition_udt_other
ON repartition_udt.udtcol = repartition_udt_other.udtcol