mirror of https://github.com/citusdata/citus.git
Merge pull request #446 from citusdata/fast_shard_pruning
Add fast shard pruning path for INSERTs on hash partitioned tablespull/1938/head
commit
552e461c48
|
@ -68,6 +68,7 @@
|
||||||
#include "distributed/citus_ruleutils.h"
|
#include "distributed/citus_ruleutils.h"
|
||||||
#include "distributed/connection_cache.h"
|
#include "distributed/connection_cache.h"
|
||||||
#include "distributed/listutils.h"
|
#include "distributed/listutils.h"
|
||||||
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
#include "distributed/master_protocol.h"
|
#include "distributed/master_protocol.h"
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
|
@ -76,6 +77,7 @@
|
||||||
#include "distributed/multi_transaction.h"
|
#include "distributed/multi_transaction.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
|
#include "distributed/shardinterval_utils.h"
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
#include "executor/execdesc.h"
|
#include "executor/execdesc.h"
|
||||||
#include "executor/executor.h"
|
#include "executor/executor.h"
|
||||||
|
@ -144,19 +146,6 @@ static void CopyToNewShards(CopyStmt *copyStatement, char *completionTag);
|
||||||
static void LockAllShards(List *shardIntervalList);
|
static void LockAllShards(List *shardIntervalList);
|
||||||
static HTAB * CreateShardConnectionHash(void);
|
static HTAB * CreateShardConnectionHash(void);
|
||||||
static int CompareShardIntervalsById(const void *leftElement, const void *rightElement);
|
static int CompareShardIntervalsById(const void *leftElement, const void *rightElement);
|
||||||
static bool IsUniformHashDistribution(ShardInterval **shardIntervalArray,
|
|
||||||
int shardCount);
|
|
||||||
static FmgrInfo * ShardIntervalCompareFunction(Var *partitionColumn, char
|
|
||||||
partitionMethod);
|
|
||||||
static ShardInterval * FindShardInterval(Datum partitionColumnValue,
|
|
||||||
ShardInterval **shardIntervalCache,
|
|
||||||
int shardCount, char partitionMethod,
|
|
||||||
FmgrInfo *compareFunction,
|
|
||||||
FmgrInfo *hashFunction, bool useBinarySearch);
|
|
||||||
static ShardInterval * SearchCachedShardInterval(Datum partitionColumnValue,
|
|
||||||
ShardInterval **shardIntervalCache,
|
|
||||||
int shardCount,
|
|
||||||
FmgrInfo *compareFunction);
|
|
||||||
static ShardConnections * GetShardConnections(HTAB *shardConnectionHash,
|
static ShardConnections * GetShardConnections(HTAB *shardConnectionHash,
|
||||||
int64 shardId,
|
int64 shardId,
|
||||||
bool *shardConnectionsFound);
|
bool *shardConnectionsFound);
|
||||||
|
@ -250,9 +239,10 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag)
|
||||||
uint32 columnCount = 0;
|
uint32 columnCount = 0;
|
||||||
Datum *columnValues = NULL;
|
Datum *columnValues = NULL;
|
||||||
bool *columnNulls = NULL;
|
bool *columnNulls = NULL;
|
||||||
TypeCacheEntry *typeEntry = NULL;
|
|
||||||
FmgrInfo *hashFunction = NULL;
|
FmgrInfo *hashFunction = NULL;
|
||||||
FmgrInfo *compareFunction = NULL;
|
FmgrInfo *compareFunction = NULL;
|
||||||
|
bool hasUniformHashDistribution = false;
|
||||||
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(tableId);
|
||||||
|
|
||||||
int shardCount = 0;
|
int shardCount = 0;
|
||||||
List *shardIntervalList = NULL;
|
List *shardIntervalList = NULL;
|
||||||
|
@ -275,12 +265,11 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag)
|
||||||
Var *partitionColumn = PartitionColumn(tableId, 0);
|
Var *partitionColumn = PartitionColumn(tableId, 0);
|
||||||
char partitionMethod = PartitionMethod(tableId);
|
char partitionMethod = PartitionMethod(tableId);
|
||||||
|
|
||||||
/* resolve hash function for partition column */
|
/* get hash function for partition column */
|
||||||
typeEntry = lookup_type_cache(partitionColumn->vartype, TYPECACHE_HASH_PROC_FINFO);
|
hashFunction = cacheEntry->hashFunction;
|
||||||
hashFunction = &(typeEntry->hash_proc_finfo);
|
|
||||||
|
|
||||||
/* resolve compare function for shard intervals */
|
/* get compare function for shard intervals */
|
||||||
compareFunction = ShardIntervalCompareFunction(partitionColumn, partitionMethod);
|
compareFunction = cacheEntry->shardIntervalCompareFunction;
|
||||||
|
|
||||||
/* allocate column values and nulls arrays */
|
/* allocate column values and nulls arrays */
|
||||||
distributedRelation = heap_open(tableId, RowExclusiveLock);
|
distributedRelation = heap_open(tableId, RowExclusiveLock);
|
||||||
|
@ -311,16 +300,26 @@ CopyToExistingShards(CopyStmt *copyStatement, char *completionTag)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* error if any shard missing min/max values */
|
||||||
|
if (cacheEntry->hasUninitializedShardInterval)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
|
||||||
|
errmsg("could not start copy"),
|
||||||
|
errdetail("Distributed relation \"%s\" has shards "
|
||||||
|
"with missing shardminvalue/shardmaxvalue.",
|
||||||
|
relationName)));
|
||||||
|
}
|
||||||
|
|
||||||
/* prevent concurrent placement changes and non-commutative DML statements */
|
/* prevent concurrent placement changes and non-commutative DML statements */
|
||||||
LockAllShards(shardIntervalList);
|
LockAllShards(shardIntervalList);
|
||||||
|
|
||||||
/* initialize the shard interval cache */
|
/* initialize the shard interval cache */
|
||||||
shardCount = list_length(shardIntervalList);
|
shardCount = cacheEntry->shardIntervalArrayLength;
|
||||||
shardIntervalCache = SortedShardIntervalArray(shardIntervalList);
|
shardIntervalCache = cacheEntry->sortedShardIntervalArray;
|
||||||
|
hasUniformHashDistribution = cacheEntry->hasUniformHashDistribution;
|
||||||
|
|
||||||
/* determine whether to use binary search */
|
/* determine whether to use binary search */
|
||||||
if (partitionMethod != DISTRIBUTE_BY_HASH ||
|
if (partitionMethod != DISTRIBUTE_BY_HASH || !hasUniformHashDistribution)
|
||||||
!IsUniformHashDistribution(shardIntervalCache, shardCount))
|
|
||||||
{
|
{
|
||||||
useBinarySearch = true;
|
useBinarySearch = true;
|
||||||
}
|
}
|
||||||
|
@ -736,164 +735,6 @@ CompareShardIntervalsById(const void *leftElement, const void *rightElement)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* ShardIntervalCompareFunction returns the appropriate compare function for the
|
|
||||||
* partition column type. In case of hash-partitioning, it always returns the compare
|
|
||||||
* function for integers.
|
|
||||||
*/
|
|
||||||
static FmgrInfo *
|
|
||||||
ShardIntervalCompareFunction(Var *partitionColumn, char partitionMethod)
|
|
||||||
{
|
|
||||||
FmgrInfo *compareFunction = NULL;
|
|
||||||
|
|
||||||
if (partitionMethod == DISTRIBUTE_BY_HASH)
|
|
||||||
{
|
|
||||||
compareFunction = GetFunctionInfo(INT4OID, BTREE_AM_OID, BTORDER_PROC);
|
|
||||||
}
|
|
||||||
else if (partitionMethod == DISTRIBUTE_BY_RANGE)
|
|
||||||
{
|
|
||||||
compareFunction = GetFunctionInfo(partitionColumn->vartype,
|
|
||||||
BTREE_AM_OID, BTORDER_PROC);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errcode(ERRCODE_INVALID_PARAMETER_VALUE),
|
|
||||||
errmsg("unsupported partition method %d", partitionMethod)));
|
|
||||||
}
|
|
||||||
|
|
||||||
return compareFunction;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* IsUniformHashDistribution determines whether the given list of sorted shards
|
|
||||||
* has a uniform hash distribution, as produced by master_create_worker_shards.
|
|
||||||
*/
|
|
||||||
static bool
|
|
||||||
IsUniformHashDistribution(ShardInterval **shardIntervalArray, int shardCount)
|
|
||||||
{
|
|
||||||
uint64 hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
|
|
||||||
int shardIndex = 0;
|
|
||||||
|
|
||||||
for (shardIndex = 0; shardIndex < shardCount; shardIndex++)
|
|
||||||
{
|
|
||||||
ShardInterval *shardInterval = shardIntervalArray[shardIndex];
|
|
||||||
int32 shardMinHashToken = INT32_MIN + (shardIndex * hashTokenIncrement);
|
|
||||||
int32 shardMaxHashToken = shardMinHashToken + (hashTokenIncrement - 1);
|
|
||||||
|
|
||||||
if (shardIndex == (shardCount - 1))
|
|
||||||
{
|
|
||||||
shardMaxHashToken = INT32_MAX;
|
|
||||||
}
|
|
||||||
|
|
||||||
if (DatumGetInt32(shardInterval->minValue) != shardMinHashToken ||
|
|
||||||
DatumGetInt32(shardInterval->maxValue) != shardMaxHashToken)
|
|
||||||
{
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* FindShardInterval finds a single shard interval in the cache for the
|
|
||||||
* given partition column value.
|
|
||||||
*/
|
|
||||||
static ShardInterval *
|
|
||||||
FindShardInterval(Datum partitionColumnValue, ShardInterval **shardIntervalCache,
|
|
||||||
int shardCount, char partitionMethod, FmgrInfo *compareFunction,
|
|
||||||
FmgrInfo *hashFunction, bool useBinarySearch)
|
|
||||||
{
|
|
||||||
ShardInterval *shardInterval = NULL;
|
|
||||||
|
|
||||||
if (partitionMethod == DISTRIBUTE_BY_HASH)
|
|
||||||
{
|
|
||||||
int hashedValue = DatumGetInt32(FunctionCall1(hashFunction,
|
|
||||||
partitionColumnValue));
|
|
||||||
if (useBinarySearch)
|
|
||||||
{
|
|
||||||
shardInterval = SearchCachedShardInterval(Int32GetDatum(hashedValue),
|
|
||||||
shardIntervalCache, shardCount,
|
|
||||||
compareFunction);
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
uint64 hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
|
|
||||||
int shardIndex = (uint32) (hashedValue - INT32_MIN) / hashTokenIncrement;
|
|
||||||
|
|
||||||
Assert(shardIndex <= shardCount);
|
|
||||||
|
|
||||||
/*
|
|
||||||
* If the shard count is not power of 2, the range of the last
|
|
||||||
* shard becomes larger than others. For that extra piece of range,
|
|
||||||
* we still need to use the last shard.
|
|
||||||
*/
|
|
||||||
if (shardIndex == shardCount)
|
|
||||||
{
|
|
||||||
shardIndex = shardCount - 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
shardInterval = shardIntervalCache[shardIndex];
|
|
||||||
}
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
shardInterval = SearchCachedShardInterval(partitionColumnValue,
|
|
||||||
shardIntervalCache, shardCount,
|
|
||||||
compareFunction);
|
|
||||||
}
|
|
||||||
|
|
||||||
return shardInterval;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* SearchCachedShardInterval performs a binary search for a shard interval matching a
|
|
||||||
* given partition column value and returns it.
|
|
||||||
*/
|
|
||||||
static ShardInterval *
|
|
||||||
SearchCachedShardInterval(Datum partitionColumnValue, ShardInterval **shardIntervalCache,
|
|
||||||
int shardCount, FmgrInfo *compareFunction)
|
|
||||||
{
|
|
||||||
int lowerBoundIndex = 0;
|
|
||||||
int upperBoundIndex = shardCount;
|
|
||||||
|
|
||||||
while (lowerBoundIndex < upperBoundIndex)
|
|
||||||
{
|
|
||||||
int middleIndex = (lowerBoundIndex + upperBoundIndex) / 2;
|
|
||||||
int maxValueComparison = 0;
|
|
||||||
int minValueComparison = 0;
|
|
||||||
|
|
||||||
minValueComparison = FunctionCall2Coll(compareFunction,
|
|
||||||
DEFAULT_COLLATION_OID,
|
|
||||||
partitionColumnValue,
|
|
||||||
shardIntervalCache[middleIndex]->minValue);
|
|
||||||
|
|
||||||
if (DatumGetInt32(minValueComparison) < 0)
|
|
||||||
{
|
|
||||||
upperBoundIndex = middleIndex;
|
|
||||||
continue;
|
|
||||||
}
|
|
||||||
|
|
||||||
maxValueComparison = FunctionCall2Coll(compareFunction,
|
|
||||||
DEFAULT_COLLATION_OID,
|
|
||||||
partitionColumnValue,
|
|
||||||
shardIntervalCache[middleIndex]->maxValue);
|
|
||||||
|
|
||||||
if (DatumGetInt32(maxValueComparison) <= 0)
|
|
||||||
{
|
|
||||||
return shardIntervalCache[middleIndex];
|
|
||||||
}
|
|
||||||
|
|
||||||
lowerBoundIndex = middleIndex + 1;
|
|
||||||
}
|
|
||||||
|
|
||||||
return NULL;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* GetShardConnections finds existing connections for a shard in the hash
|
* GetShardConnections finds existing connections for a shard in the hash
|
||||||
* or opens new connections to each active placement and starts a (binary) COPY
|
* or opens new connections to each active placement and starts a (binary) COPY
|
||||||
|
|
|
@ -46,6 +46,9 @@ static uint64 * AllocateUint64(uint64 value);
|
||||||
* LoadShardIntervalList returns a list of shard intervals related for a given
|
* LoadShardIntervalList returns a list of shard intervals related for a given
|
||||||
* distributed table. The function returns an empty list if no shards can be
|
* distributed table. The function returns an empty list if no shards can be
|
||||||
* found for the given relation.
|
* found for the given relation.
|
||||||
|
* Since LoadShardIntervalList relies on sortedShardIntervalArray, it returns
|
||||||
|
* a shard interval list whose elements are sorted on shardminvalue. Shard intervals
|
||||||
|
* with uninitialized shard min/max values are placed in the end of the list.
|
||||||
*/
|
*/
|
||||||
List *
|
List *
|
||||||
LoadShardIntervalList(Oid relationId)
|
LoadShardIntervalList(Oid relationId)
|
||||||
|
@ -59,7 +62,7 @@ LoadShardIntervalList(Oid relationId)
|
||||||
ShardInterval *newShardInterval = NULL;
|
ShardInterval *newShardInterval = NULL;
|
||||||
newShardInterval = (ShardInterval *) palloc0(sizeof(ShardInterval));
|
newShardInterval = (ShardInterval *) palloc0(sizeof(ShardInterval));
|
||||||
|
|
||||||
CopyShardInterval(&cacheEntry->shardIntervalArray[i], newShardInterval);
|
CopyShardInterval(cacheEntry->sortedShardIntervalArray[i], newShardInterval);
|
||||||
|
|
||||||
shardList = lappend(shardList, newShardInterval);
|
shardList = lappend(shardList, newShardInterval);
|
||||||
}
|
}
|
||||||
|
@ -91,6 +94,9 @@ ShardIntervalCount(Oid relationId)
|
||||||
/*
|
/*
|
||||||
* LoadShardList reads list of shards for given relationId from pg_dist_shard,
|
* LoadShardList reads list of shards for given relationId from pg_dist_shard,
|
||||||
* and returns the list of found shardIds.
|
* and returns the list of found shardIds.
|
||||||
|
* Since LoadShardList relies on sortedShardIntervalArray, it returns a shard
|
||||||
|
* list whose elements are sorted on shardminvalue. Shards with uninitialized
|
||||||
|
* shard min/max values are placed in the end of the list.
|
||||||
*/
|
*/
|
||||||
List *
|
List *
|
||||||
LoadShardList(Oid relationId)
|
LoadShardList(Oid relationId)
|
||||||
|
@ -101,7 +107,7 @@ LoadShardList(Oid relationId)
|
||||||
|
|
||||||
for (i = 0; i < cacheEntry->shardIntervalArrayLength; i++)
|
for (i = 0; i < cacheEntry->shardIntervalArrayLength; i++)
|
||||||
{
|
{
|
||||||
ShardInterval *currentShardInterval = &cacheEntry->shardIntervalArray[i];
|
ShardInterval *currentShardInterval = cacheEntry->sortedShardIntervalArray[i];
|
||||||
uint64 *shardIdPointer = AllocateUint64(currentShardInterval->shardId);
|
uint64 *shardIdPointer = AllocateUint64(currentShardInterval->shardId);
|
||||||
|
|
||||||
shardList = lappend(shardList, shardIdPointer);
|
shardList = lappend(shardList, shardIdPointer);
|
||||||
|
|
|
@ -56,7 +56,6 @@ static List * MergeShardIntervals(List *leftShardIntervalList,
|
||||||
List *rightShardIntervalList, JoinType joinType);
|
List *rightShardIntervalList, JoinType joinType);
|
||||||
static bool ShardIntervalsMatch(List *leftShardIntervalList,
|
static bool ShardIntervalsMatch(List *leftShardIntervalList,
|
||||||
List *rightShardIntervalList);
|
List *rightShardIntervalList);
|
||||||
static List * LoadSortedShardIntervalList(Oid relationId);
|
|
||||||
static List * JoinOrderForTable(TableEntry *firstTable, List *tableEntryList,
|
static List * JoinOrderForTable(TableEntry *firstTable, List *tableEntryList,
|
||||||
List *joinClauseList);
|
List *joinClauseList);
|
||||||
static List * BestJoinOrder(List *candidateJoinOrders);
|
static List * BestJoinOrder(List *candidateJoinOrders);
|
||||||
|
@ -123,6 +122,22 @@ FixedJoinOrderList(FromExpr *fromExpr, List *tableEntryList)
|
||||||
List *joinedTableList = NIL;
|
List *joinedTableList = NIL;
|
||||||
JoinOrderNode *firstJoinNode = NULL;
|
JoinOrderNode *firstJoinNode = NULL;
|
||||||
JoinOrderNode *currentJoinNode = NULL;
|
JoinOrderNode *currentJoinNode = NULL;
|
||||||
|
ListCell *tableEntryCell = NULL;
|
||||||
|
|
||||||
|
foreach(tableEntryCell, tableEntryList)
|
||||||
|
{
|
||||||
|
TableEntry *rangeTableEntry = (TableEntry *) lfirst(tableEntryCell);
|
||||||
|
Oid relationId = rangeTableEntry->relationId;
|
||||||
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(relationId);
|
||||||
|
|
||||||
|
if (cacheEntry->hasUninitializedShardInterval)
|
||||||
|
{
|
||||||
|
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 shard min/max values.")));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/* get the FROM section as a flattened list of JoinExpr nodes */
|
/* get the FROM section as a flattened list of JoinExpr nodes */
|
||||||
joinList = JoinExprList(fromExpr);
|
joinList = JoinExprList(fromExpr);
|
||||||
|
@ -159,8 +174,8 @@ FixedJoinOrderList(FromExpr *fromExpr, List *tableEntryList)
|
||||||
joinClauseList = list_concat(joinClauseList, joinWhereClauseList);
|
joinClauseList = list_concat(joinClauseList, joinWhereClauseList);
|
||||||
}
|
}
|
||||||
|
|
||||||
/* get the list of shards to check broadcast/local join possibility */
|
/* get the sorted list of shards to check broadcast/local join possibility */
|
||||||
candidateShardList = LoadSortedShardIntervalList(nextTable->relationId);
|
candidateShardList = LoadShardIntervalList(nextTable->relationId);
|
||||||
|
|
||||||
/* find the best join rule type */
|
/* find the best join rule type */
|
||||||
nextJoinNode = EvaluateJoinRules(joinedTableList, currentJoinNode,
|
nextJoinNode = EvaluateJoinRules(joinedTableList, currentJoinNode,
|
||||||
|
@ -268,8 +283,7 @@ CreateFirstJoinOrderNode(FromExpr *fromExpr, List *tableEntryList)
|
||||||
firstPartitionColumn,
|
firstPartitionColumn,
|
||||||
firstPartitionMethod);
|
firstPartitionMethod);
|
||||||
|
|
||||||
firstJoinNode->shardIntervalList =
|
firstJoinNode->shardIntervalList = LoadShardIntervalList(firstTable->relationId);
|
||||||
LoadSortedShardIntervalList(firstTable->relationId);
|
|
||||||
|
|
||||||
return firstJoinNode;
|
return firstJoinNode;
|
||||||
}
|
}
|
||||||
|
@ -462,40 +476,6 @@ MergeShardIntervals(List *leftShardIntervalList, List *rightShardIntervalList,
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* LoadSortedShardIntervalList loads a list of shard intervals from the metadata
|
|
||||||
* and sorts the list by the minimum value of the intervals.
|
|
||||||
*/
|
|
||||||
static List *
|
|
||||||
LoadSortedShardIntervalList(Oid relationId)
|
|
||||||
{
|
|
||||||
List *shardIntervalList = NIL;
|
|
||||||
int shardCount = 0;
|
|
||||||
int intervalIndex = 0;
|
|
||||||
ShardInterval **sortedShardIntervalArray = NULL;
|
|
||||||
List *sortedShardIntervalList = NIL;
|
|
||||||
|
|
||||||
shardIntervalList = LoadShardIntervalList(relationId);
|
|
||||||
|
|
||||||
shardCount = list_length(shardIntervalList);
|
|
||||||
if (shardCount <= 1)
|
|
||||||
{
|
|
||||||
return shardIntervalList;
|
|
||||||
}
|
|
||||||
|
|
||||||
sortedShardIntervalArray = SortedShardIntervalArray(shardIntervalList);
|
|
||||||
|
|
||||||
for (intervalIndex = 0; intervalIndex < shardCount; intervalIndex++)
|
|
||||||
{
|
|
||||||
ShardInterval *shardInterval = sortedShardIntervalArray[intervalIndex];
|
|
||||||
|
|
||||||
sortedShardIntervalList = lappend(sortedShardIntervalList, shardInterval);
|
|
||||||
}
|
|
||||||
|
|
||||||
return sortedShardIntervalList;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* JoinOnColumns determines whether two columns are joined by a given join clause
|
* JoinOnColumns determines whether two columns are joined by a given join clause
|
||||||
* list.
|
* list.
|
||||||
|
|
|
@ -26,6 +26,7 @@
|
||||||
#include "catalog/pg_type.h"
|
#include "catalog/pg_type.h"
|
||||||
#include "commands/extension.h"
|
#include "commands/extension.h"
|
||||||
#include "distributed/citus_nodes.h"
|
#include "distributed/citus_nodes.h"
|
||||||
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/multi_logical_optimizer.h"
|
#include "distributed/multi_logical_optimizer.h"
|
||||||
#include "distributed/multi_logical_planner.h"
|
#include "distributed/multi_logical_planner.h"
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
|
@ -137,7 +138,7 @@ static bool SupportedLateralQuery(Query *parentQuery, Query *lateralQuery);
|
||||||
static bool JoinOnPartitionColumn(Query *query);
|
static bool JoinOnPartitionColumn(Query *query);
|
||||||
static void ErrorIfUnsupportedShardDistribution(Query *query);
|
static void ErrorIfUnsupportedShardDistribution(Query *query);
|
||||||
static List * RelationIdList(Query *query);
|
static List * RelationIdList(Query *query);
|
||||||
static bool CoPartitionedTables(List *firstShardList, List *secondShardList);
|
static bool CoPartitionedTables(Oid firstRelationId, Oid secondRelationId);
|
||||||
static bool ShardIntervalsEqual(ShardInterval *firstInterval,
|
static bool ShardIntervalsEqual(ShardInterval *firstInterval,
|
||||||
ShardInterval *secondInterval);
|
ShardInterval *secondInterval);
|
||||||
static void ErrorIfUnsupportedFilters(Query *subquery);
|
static void ErrorIfUnsupportedFilters(Query *subquery);
|
||||||
|
@ -2310,8 +2311,8 @@ TablePartitioningSupportsDistinct(List *tableNodeList, MultiExtendedOp *opNode,
|
||||||
*/
|
*/
|
||||||
partitionMethod = PartitionMethod(relationId);
|
partitionMethod = PartitionMethod(relationId);
|
||||||
|
|
||||||
if (partitionMethod == DISTRIBUTE_BY_RANGE
|
if (partitionMethod == DISTRIBUTE_BY_RANGE ||
|
||||||
|| partitionMethod == DISTRIBUTE_BY_HASH)
|
partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
{
|
{
|
||||||
Var *tablePartitionColumn = tableNode->partitionColumn;
|
Var *tablePartitionColumn = tableNode->partitionColumn;
|
||||||
bool groupedByPartitionColumn = false;
|
bool groupedByPartitionColumn = false;
|
||||||
|
@ -3386,7 +3387,7 @@ JoinOnPartitionColumn(Query *query)
|
||||||
static void
|
static void
|
||||||
ErrorIfUnsupportedShardDistribution(Query *query)
|
ErrorIfUnsupportedShardDistribution(Query *query)
|
||||||
{
|
{
|
||||||
List *firstShardIntervalList = NIL;
|
Oid firstTableRelationId = InvalidOid;
|
||||||
List *relationIdList = RelationIdList(query);
|
List *relationIdList = RelationIdList(query);
|
||||||
ListCell *relationIdCell = NULL;
|
ListCell *relationIdCell = NULL;
|
||||||
uint32 relationIndex = 0;
|
uint32 relationIndex = 0;
|
||||||
|
@ -3425,21 +3426,21 @@ ErrorIfUnsupportedShardDistribution(Query *query)
|
||||||
foreach(relationIdCell, relationIdList)
|
foreach(relationIdCell, relationIdList)
|
||||||
{
|
{
|
||||||
Oid relationId = lfirst_oid(relationIdCell);
|
Oid relationId = lfirst_oid(relationIdCell);
|
||||||
List *currentShardIntervalList = LoadShardIntervalList(relationId);
|
|
||||||
bool coPartitionedTables = false;
|
bool coPartitionedTables = false;
|
||||||
|
Oid currentRelationId = relationId;
|
||||||
|
|
||||||
/* get shard list of first relation and continue for the next relation */
|
/* get shard list of first relation and continue for the next relation */
|
||||||
if (relationIndex == 0)
|
if (relationIndex == 0)
|
||||||
{
|
{
|
||||||
firstShardIntervalList = currentShardIntervalList;
|
firstTableRelationId = relationId;
|
||||||
relationIndex++;
|
relationIndex++;
|
||||||
|
|
||||||
continue;
|
continue;
|
||||||
}
|
}
|
||||||
|
|
||||||
/* check if this table has 1-1 shard partitioning with first table */
|
/* check if this table has 1-1 shard partitioning with first table */
|
||||||
coPartitionedTables = CoPartitionedTables(firstShardIntervalList,
|
coPartitionedTables = CoPartitionedTables(firstTableRelationId,
|
||||||
currentShardIntervalList);
|
currentRelationId);
|
||||||
if (!coPartitionedTables)
|
if (!coPartitionedTables)
|
||||||
{
|
{
|
||||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||||
|
@ -3478,21 +3479,23 @@ RelationIdList(Query *query)
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* CoPartitionedTables checks if given shard lists have 1-to-1 shard partitioning.
|
* CoPartitionedTables checks if given two distributed tables have 1-to-1 shard
|
||||||
* It first sorts both list according to shard interval minimum values. Then it
|
* partitioning. It uses shard interval array that are sorted on interval minimum
|
||||||
* compares every shard interval in order and if any pair of shard intervals are
|
* values. Then it compares every shard interval in order and if any pair of
|
||||||
* not equal it returns false.
|
* shard intervals are not equal it returns false.
|
||||||
*/
|
*/
|
||||||
static bool
|
static bool
|
||||||
CoPartitionedTables(List *firstShardList, List *secondShardList)
|
CoPartitionedTables(Oid firstRelationId, Oid secondRelationId)
|
||||||
{
|
{
|
||||||
bool coPartitionedTables = true;
|
bool coPartitionedTables = true;
|
||||||
uint32 intervalIndex = 0;
|
uint32 intervalIndex = 0;
|
||||||
ShardInterval **sortedFirstIntervalArray = NULL;
|
DistTableCacheEntry *firstTableCache = DistributedTableCacheEntry(firstRelationId);
|
||||||
ShardInterval **sortedSecondIntervalArray = NULL;
|
DistTableCacheEntry *secondTableCache = DistributedTableCacheEntry(secondRelationId);
|
||||||
|
ShardInterval **sortedFirstIntervalArray = firstTableCache->sortedShardIntervalArray;
|
||||||
uint32 firstListShardCount = list_length(firstShardList);
|
ShardInterval **sortedSecondIntervalArray =
|
||||||
uint32 secondListShardCount = list_length(secondShardList);
|
secondTableCache->sortedShardIntervalArray;
|
||||||
|
uint32 firstListShardCount = firstTableCache->shardIntervalArrayLength;
|
||||||
|
uint32 secondListShardCount = secondTableCache->shardIntervalArrayLength;
|
||||||
|
|
||||||
if (firstListShardCount != secondListShardCount)
|
if (firstListShardCount != secondListShardCount)
|
||||||
{
|
{
|
||||||
|
@ -3505,9 +3508,6 @@ CoPartitionedTables(List *firstShardList, List *secondShardList)
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
sortedFirstIntervalArray = SortedShardIntervalArray(firstShardList);
|
|
||||||
sortedSecondIntervalArray = SortedShardIntervalArray(secondShardList);
|
|
||||||
|
|
||||||
for (intervalIndex = 0; intervalIndex < firstListShardCount; intervalIndex++)
|
for (intervalIndex = 0; intervalIndex < firstListShardCount; intervalIndex++)
|
||||||
{
|
{
|
||||||
ShardInterval *firstInterval = sortedFirstIntervalArray[intervalIndex];
|
ShardInterval *firstInterval = sortedFirstIntervalArray[intervalIndex];
|
||||||
|
|
|
@ -37,6 +37,7 @@
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
#include "distributed/pg_dist_shard.h"
|
#include "distributed/pg_dist_shard.h"
|
||||||
|
#include "distributed/shardinterval_utils.h"
|
||||||
#include "distributed/task_tracker.h"
|
#include "distributed/task_tracker.h"
|
||||||
#include "distributed/worker_manager.h"
|
#include "distributed/worker_manager.h"
|
||||||
#include "distributed/worker_protocol.h"
|
#include "distributed/worker_protocol.h"
|
||||||
|
@ -110,8 +111,6 @@ static MapMergeJob * BuildMapMergeJob(Query *jobQuery, List *dependedJobList,
|
||||||
Oid baseRelationId,
|
Oid baseRelationId,
|
||||||
BoundaryNodeJobType boundaryNodeJobType);
|
BoundaryNodeJobType boundaryNodeJobType);
|
||||||
static uint32 HashPartitionCount(void);
|
static uint32 HashPartitionCount(void);
|
||||||
static int CompareShardIntervals(const void *leftElement, const void *rightElement,
|
|
||||||
FmgrInfo *typeCompareFunction);
|
|
||||||
static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
|
static ArrayType * SplitPointObject(ShardInterval **shardIntervalArray,
|
||||||
uint32 shardIntervalCount);
|
uint32 shardIntervalCount);
|
||||||
|
|
||||||
|
@ -1716,10 +1715,17 @@ BuildMapMergeJob(Query *jobQuery, List *dependedJobList, Var *partitionKey,
|
||||||
else if (partitionType == RANGE_PARTITION_TYPE)
|
else if (partitionType == RANGE_PARTITION_TYPE)
|
||||||
{
|
{
|
||||||
/* build the split point object for the table on the right-hand side */
|
/* build the split point object for the table on the right-hand side */
|
||||||
List *shardIntervalList = LoadShardIntervalList(baseRelationId);
|
DistTableCacheEntry *cache = DistributedTableCacheEntry(baseRelationId);
|
||||||
uint32 shardCount = (uint32) list_length(shardIntervalList);
|
bool hasUninitializedShardInterval = false;
|
||||||
ShardInterval **sortedShardIntervalArray =
|
uint32 shardCount = cache->shardIntervalArrayLength;
|
||||||
SortedShardIntervalArray(shardIntervalList);
|
ShardInterval **sortedShardIntervalArray = cache->sortedShardIntervalArray;
|
||||||
|
|
||||||
|
hasUninitializedShardInterval = cache->hasUninitializedShardInterval;
|
||||||
|
if (hasUninitializedShardInterval)
|
||||||
|
{
|
||||||
|
ereport(ERROR, (errmsg("cannot range repartition shard with "
|
||||||
|
"missing min/max values")));
|
||||||
|
}
|
||||||
|
|
||||||
/* this join-type currently doesn't work for hash partitioned tables */
|
/* this join-type currently doesn't work for hash partitioned tables */
|
||||||
char basePartitionMethod PG_USED_FOR_ASSERTS_ONLY =
|
char basePartitionMethod PG_USED_FOR_ASSERTS_ONLY =
|
||||||
|
@ -1754,78 +1760,6 @@ HashPartitionCount(void)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* SortedShardIntervalArray returns a sorted array of shard intervals for shards
|
|
||||||
* in the given shard list. The array elements are sorted in in ascending order
|
|
||||||
* according to shard interval's minimum value.
|
|
||||||
*/
|
|
||||||
ShardInterval **
|
|
||||||
SortedShardIntervalArray(List *shardIntervalList)
|
|
||||||
{
|
|
||||||
FmgrInfo *typeCompareFunction = NULL;
|
|
||||||
ListCell *shardIntervalCell = NULL;
|
|
||||||
uint32 shardIntervalIndex = 0;
|
|
||||||
|
|
||||||
ShardInterval **shardIntervalArray = NULL;
|
|
||||||
uint32 shardIntervalCount = (uint32) list_length(shardIntervalList);
|
|
||||||
Assert(shardIntervalCount > 0);
|
|
||||||
|
|
||||||
/* allocate an array for sorted shard intervals */
|
|
||||||
shardIntervalArray = palloc0(shardIntervalCount * sizeof(ShardInterval *));
|
|
||||||
|
|
||||||
/* fill in the array with shard intervals */
|
|
||||||
foreach(shardIntervalCell, shardIntervalList)
|
|
||||||
{
|
|
||||||
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
|
|
||||||
if (shardInterval->minValueExists && shardInterval->maxValueExists)
|
|
||||||
{
|
|
||||||
shardIntervalArray[shardIntervalIndex] = shardInterval;
|
|
||||||
}
|
|
||||||
else
|
|
||||||
{
|
|
||||||
ereport(ERROR, (errmsg("cannot range repartition shard " UINT64_FORMAT
|
|
||||||
" with missing min/max values",
|
|
||||||
shardInterval->shardId)));
|
|
||||||
}
|
|
||||||
|
|
||||||
/* resolve the datum type and comparison function on first pass */
|
|
||||||
if (shardIntervalIndex == 0)
|
|
||||||
{
|
|
||||||
Oid typeId = shardInterval->valueTypeId;
|
|
||||||
typeCompareFunction = GetFunctionInfo(typeId, BTREE_AM_OID, BTORDER_PROC);
|
|
||||||
}
|
|
||||||
shardIntervalIndex++;
|
|
||||||
}
|
|
||||||
|
|
||||||
/* sort shard intervals by their minimum values in ascending order */
|
|
||||||
qsort_arg(shardIntervalArray, shardIntervalCount, sizeof(ShardInterval *),
|
|
||||||
(qsort_arg_comparator) CompareShardIntervals, (void *) typeCompareFunction);
|
|
||||||
|
|
||||||
return shardIntervalArray;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
|
||||||
* CompareShardIntervals acts as a helper function to compare two shard interval
|
|
||||||
* pointers by their minimum values, using the value's type comparison function.
|
|
||||||
*/
|
|
||||||
static int
|
|
||||||
CompareShardIntervals(const void *leftElement, const void *rightElement,
|
|
||||||
FmgrInfo *typeCompareFunction)
|
|
||||||
{
|
|
||||||
ShardInterval **leftShardInterval = (ShardInterval **) leftElement;
|
|
||||||
ShardInterval **rightShardInterval = (ShardInterval **) rightElement;
|
|
||||||
|
|
||||||
Datum leftDatum = (*leftShardInterval)->minValue;
|
|
||||||
Datum rightDatum = (*rightShardInterval)->minValue;
|
|
||||||
|
|
||||||
Datum comparisonDatum = CompareCall2(typeCompareFunction, leftDatum, rightDatum);
|
|
||||||
int comparisonResult = DatumGetInt32(comparisonDatum);
|
|
||||||
|
|
||||||
return comparisonResult;
|
|
||||||
}
|
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* SplitPointObject walks over shard intervals in the given array, extracts each
|
* SplitPointObject walks over shard intervals in the given array, extracts each
|
||||||
* shard interval's minimum value, sorts and inserts these minimum values into a
|
* shard interval's minimum value, sorts and inserts these minimum values into a
|
||||||
|
@ -2031,10 +1965,9 @@ SubquerySqlTaskList(Job *job)
|
||||||
List *shardIntervalList = LoadShardIntervalList(relationId);
|
List *shardIntervalList = LoadShardIntervalList(relationId);
|
||||||
List *finalShardIntervalList = NIL;
|
List *finalShardIntervalList = NIL;
|
||||||
ListCell *fragmentCombinationCell = NULL;
|
ListCell *fragmentCombinationCell = NULL;
|
||||||
ShardInterval **sortedIntervalArray = NULL;
|
ListCell *shardIntervalCell = NULL;
|
||||||
uint32 tableId = rangeTableIndex + 1; /* tableId starts from 1 */
|
uint32 tableId = rangeTableIndex + 1; /* tableId starts from 1 */
|
||||||
uint32 finalShardCount = 0;
|
uint32 finalShardCount = 0;
|
||||||
uint32 shardIndex = 0;
|
|
||||||
|
|
||||||
if (opExpressionList != NIL)
|
if (opExpressionList != NIL)
|
||||||
{
|
{
|
||||||
|
@ -2056,12 +1989,11 @@ SubquerySqlTaskList(Job *job)
|
||||||
return NIL;
|
return NIL;
|
||||||
}
|
}
|
||||||
|
|
||||||
sortedIntervalArray = SortedShardIntervalArray(finalShardIntervalList);
|
|
||||||
fragmentCombinationCell = list_head(fragmentCombinationList);
|
fragmentCombinationCell = list_head(fragmentCombinationList);
|
||||||
|
|
||||||
for (shardIndex = 0; shardIndex < finalShardCount; shardIndex++)
|
foreach(shardIntervalCell, finalShardIntervalList)
|
||||||
{
|
{
|
||||||
ShardInterval *shardInterval = sortedIntervalArray[shardIndex];
|
ShardInterval *shardInterval = (ShardInterval *) lfirst(shardIntervalCell);
|
||||||
|
|
||||||
RangeTableFragment *shardFragment = palloc0(fragmentSize);
|
RangeTableFragment *shardFragment = palloc0(fragmentSize);
|
||||||
shardFragment->fragmentReference = &(shardInterval->shardId);
|
shardFragment->fragmentReference = &(shardInterval->shardId);
|
||||||
|
|
|
@ -33,6 +33,7 @@
|
||||||
#include "distributed/citus_ruleutils.h"
|
#include "distributed/citus_ruleutils.h"
|
||||||
#include "distributed/relay_utility.h"
|
#include "distributed/relay_utility.h"
|
||||||
#include "distributed/resource_lock.h"
|
#include "distributed/resource_lock.h"
|
||||||
|
#include "distributed/shardinterval_utils.h"
|
||||||
#include "executor/execdesc.h"
|
#include "executor/execdesc.h"
|
||||||
#include "lib/stringinfo.h"
|
#include "lib/stringinfo.h"
|
||||||
#if (PG_VERSION_NUM >= 90500)
|
#if (PG_VERSION_NUM >= 90500)
|
||||||
|
@ -51,6 +52,7 @@
|
||||||
#include "utils/lsyscache.h"
|
#include "utils/lsyscache.h"
|
||||||
#include "utils/rel.h"
|
#include "utils/rel.h"
|
||||||
#include "utils/relcache.h"
|
#include "utils/relcache.h"
|
||||||
|
#include "utils/typcache.h"
|
||||||
|
|
||||||
|
|
||||||
/* planner functions forward declarations */
|
/* planner functions forward declarations */
|
||||||
|
@ -62,8 +64,11 @@ static OnConflictExpr * RebuildOnConflict(Oid relationId,
|
||||||
#endif
|
#endif
|
||||||
static ShardInterval * TargetShardInterval(Query *query);
|
static ShardInterval * TargetShardInterval(Query *query);
|
||||||
static List * QueryRestrictList(Query *query);
|
static List * QueryRestrictList(Query *query);
|
||||||
|
static bool FastShardPruningPossible(CmdType commandType, char partitionMethod);
|
||||||
|
static ShardInterval * FastShardPruning(Oid distributedTableId,
|
||||||
|
Const *partionColumnValue);
|
||||||
static Oid ExtractFirstDistributedTableId(Query *query);
|
static Oid ExtractFirstDistributedTableId(Query *query);
|
||||||
static Const * ExtractPartitionValue(Query *query, Var *partitionColumn);
|
static Const * ExtractInsertPartitionValue(Query *query, Var *partitionColumn);
|
||||||
static Task * RouterSelectTask(Query *query);
|
static Task * RouterSelectTask(Query *query);
|
||||||
static Job * RouterQueryJob(Query *query, Task *task);
|
static Job * RouterQueryJob(Query *query, Task *task);
|
||||||
static bool ColumnMatchExpressionAtTopLevelConjunction(Node *node, Var *column);
|
static bool ColumnMatchExpressionAtTopLevelConjunction(Node *node, Var *column);
|
||||||
|
@ -507,18 +512,19 @@ TargetShardInterval(Query *query)
|
||||||
{
|
{
|
||||||
CmdType commandType = query->commandType;
|
CmdType commandType = query->commandType;
|
||||||
bool selectTask = (commandType == CMD_SELECT);
|
bool selectTask = (commandType == CMD_SELECT);
|
||||||
List *restrictClauseList = NIL;
|
|
||||||
List *prunedShardList = NIL;
|
List *prunedShardList = NIL;
|
||||||
Index tableId = 1;
|
|
||||||
int prunedShardCount = 0;
|
int prunedShardCount = 0;
|
||||||
|
|
||||||
|
|
||||||
|
int shardCount = 0;
|
||||||
Oid distributedTableId = ExtractFirstDistributedTableId(query);
|
Oid distributedTableId = ExtractFirstDistributedTableId(query);
|
||||||
List *shardIntervalList = NIL;
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedTableId);
|
||||||
|
char partitionMethod = cacheEntry->partitionMethod;
|
||||||
|
bool fastShardPruningPossible = false;
|
||||||
|
|
||||||
/* error out if no shards exist for the table */
|
/* error out if no shards exist for the table */
|
||||||
shardIntervalList = LoadShardIntervalList(distributedTableId);
|
shardCount = cacheEntry->shardIntervalArrayLength;
|
||||||
if (shardIntervalList == NIL)
|
if (shardCount == 0)
|
||||||
{
|
{
|
||||||
char *relationName = get_rel_name(distributedTableId);
|
char *relationName = get_rel_name(distributedTableId);
|
||||||
|
|
||||||
|
@ -530,9 +536,31 @@ TargetShardInterval(Query *query)
|
||||||
"and try again.")));
|
"and try again.")));
|
||||||
}
|
}
|
||||||
|
|
||||||
restrictClauseList = QueryRestrictList(query);
|
fastShardPruningPossible = FastShardPruningPossible(query->commandType,
|
||||||
prunedShardList = PruneShardList(distributedTableId, tableId, restrictClauseList,
|
partitionMethod);
|
||||||
shardIntervalList);
|
if (fastShardPruningPossible)
|
||||||
|
{
|
||||||
|
uint32 rangeTableId = 1;
|
||||||
|
Var *partitionColumn = PartitionColumn(distributedTableId, rangeTableId);
|
||||||
|
Const *partitionValue = ExtractInsertPartitionValue(query, partitionColumn);
|
||||||
|
ShardInterval *shardInterval = FastShardPruning(distributedTableId,
|
||||||
|
partitionValue);
|
||||||
|
|
||||||
|
if (shardInterval != NULL)
|
||||||
|
{
|
||||||
|
prunedShardList = lappend(prunedShardList, shardInterval);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
List *restrictClauseList = QueryRestrictList(query);
|
||||||
|
Index tableId = 1;
|
||||||
|
List *shardIntervalList = LoadShardIntervalList(distributedTableId);
|
||||||
|
|
||||||
|
prunedShardList = PruneShardList(distributedTableId, tableId, restrictClauseList,
|
||||||
|
shardIntervalList);
|
||||||
|
}
|
||||||
|
|
||||||
prunedShardCount = list_length(prunedShardList);
|
prunedShardCount = list_length(prunedShardList);
|
||||||
if (prunedShardCount != 1)
|
if (prunedShardCount != 1)
|
||||||
{
|
{
|
||||||
|
@ -554,6 +582,74 @@ TargetShardInterval(Query *query)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* UseFastShardPruning returns true if the commandType is INSERT and partition method
|
||||||
|
* is hash or range.
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
FastShardPruningPossible(CmdType commandType, char partitionMethod)
|
||||||
|
{
|
||||||
|
/* we currently only support INSERTs */
|
||||||
|
if (commandType != CMD_INSERT)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* fast shard pruning is only supported for hash and range partitioned tables */
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_HASH || partitionMethod == DISTRIBUTE_BY_RANGE)
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* FastShardPruning is a higher level API for FindShardInterval function. Given the relationId
|
||||||
|
* of the distributed table and partitionValue, FastShardPruning function finds the corresponding
|
||||||
|
* shard interval that the partitionValue should be in. FastShardPruning returns NULL if no
|
||||||
|
* ShardIntervals exist for the given partitionValue.
|
||||||
|
*/
|
||||||
|
static ShardInterval *
|
||||||
|
FastShardPruning(Oid distributedTableId, Const *partitionValue)
|
||||||
|
{
|
||||||
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedTableId);
|
||||||
|
int shardCount = cacheEntry->shardIntervalArrayLength;
|
||||||
|
ShardInterval **sortedShardIntervalArray = cacheEntry->sortedShardIntervalArray;
|
||||||
|
bool useBinarySearch = false;
|
||||||
|
char partitionMethod = cacheEntry->partitionMethod;
|
||||||
|
FmgrInfo *shardIntervalCompareFunction = cacheEntry->shardIntervalCompareFunction;
|
||||||
|
bool hasUniformHashDistribution = cacheEntry->hasUniformHashDistribution;
|
||||||
|
FmgrInfo *hashFunction = NULL;
|
||||||
|
ShardInterval *shardInterval = NULL;
|
||||||
|
|
||||||
|
/* determine whether to use binary search */
|
||||||
|
if (partitionMethod != DISTRIBUTE_BY_HASH || !hasUniformHashDistribution)
|
||||||
|
{
|
||||||
|
useBinarySearch = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* we only need hash functions for hash distributed tables */
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
hashFunction = cacheEntry->hashFunction;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Call FindShardInterval to find the corresponding shard interval for the
|
||||||
|
* given partition value.
|
||||||
|
*/
|
||||||
|
shardInterval = FindShardInterval(partitionValue->constvalue,
|
||||||
|
sortedShardIntervalArray, shardCount,
|
||||||
|
partitionMethod,
|
||||||
|
shardIntervalCompareFunction, hashFunction,
|
||||||
|
useBinarySearch);
|
||||||
|
|
||||||
|
return shardInterval;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* QueryRestrictList returns the restriction clauses for the query. For a SELECT
|
* QueryRestrictList returns the restriction clauses for the query. For a SELECT
|
||||||
* statement these are the where-clause expressions. For INSERT statements we
|
* statement these are the where-clause expressions. For INSERT statements we
|
||||||
|
@ -572,7 +668,7 @@ QueryRestrictList(Query *query)
|
||||||
Oid distributedTableId = ExtractFirstDistributedTableId(query);
|
Oid distributedTableId = ExtractFirstDistributedTableId(query);
|
||||||
uint32 rangeTableId = 1;
|
uint32 rangeTableId = 1;
|
||||||
Var *partitionColumn = PartitionColumn(distributedTableId, rangeTableId);
|
Var *partitionColumn = PartitionColumn(distributedTableId, rangeTableId);
|
||||||
Const *partitionValue = ExtractPartitionValue(query, partitionColumn);
|
Const *partitionValue = ExtractInsertPartitionValue(query, partitionColumn);
|
||||||
|
|
||||||
OpExpr *equalityExpr = MakeOpExpression(partitionColumn, BTEqualStrategyNumber);
|
OpExpr *equalityExpr = MakeOpExpression(partitionColumn, BTEqualStrategyNumber);
|
||||||
|
|
||||||
|
@ -628,11 +724,11 @@ ExtractFirstDistributedTableId(Query *query)
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* ExtractPartitionValue extracts the partition column value from a the target
|
* ExtractPartitionValue extracts the partition column value from a the target
|
||||||
* of a modification command. If a partition value is missing altogether or is
|
* of an INSERT command. If a partition value is missing altogether or is
|
||||||
* NULL, this function throws an error.
|
* NULL, this function throws an error.
|
||||||
*/
|
*/
|
||||||
static Const *
|
static Const *
|
||||||
ExtractPartitionValue(Query *query, Var *partitionColumn)
|
ExtractInsertPartitionValue(Query *query, Var *partitionColumn)
|
||||||
{
|
{
|
||||||
Const *partitionValue = NULL;
|
Const *partitionValue = NULL;
|
||||||
TargetEntry *targetEntry = get_tle_by_resno(query->targetList,
|
TargetEntry *targetEntry = get_tle_by_resno(query->targetList,
|
||||||
|
|
|
@ -22,6 +22,7 @@
|
||||||
#include "access/skey.h"
|
#include "access/skey.h"
|
||||||
#endif
|
#endif
|
||||||
#include "catalog/pg_type.h"
|
#include "catalog/pg_type.h"
|
||||||
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
#include "distributed/multi_join_order.h"
|
#include "distributed/multi_join_order.h"
|
||||||
#include "distributed/multi_physical_planner.h"
|
#include "distributed/multi_physical_planner.h"
|
||||||
|
@ -38,6 +39,7 @@
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
static Expr * MakeTextPartitionExpression(Oid distributedTableId, text *value);
|
static Expr * MakeTextPartitionExpression(Oid distributedTableId, text *value);
|
||||||
static ArrayType * PrunedShardIdsForTable(Oid distributedTableId, List *whereClauseList);
|
static ArrayType * PrunedShardIdsForTable(Oid distributedTableId, List *whereClauseList);
|
||||||
|
static ArrayType * SortedShardIntervalArray(Oid distributedTableId);
|
||||||
|
|
||||||
|
|
||||||
/* declarations for dynamic loading */
|
/* declarations for dynamic loading */
|
||||||
|
@ -46,6 +48,7 @@ PG_FUNCTION_INFO_V1(prune_using_single_value);
|
||||||
PG_FUNCTION_INFO_V1(prune_using_either_value);
|
PG_FUNCTION_INFO_V1(prune_using_either_value);
|
||||||
PG_FUNCTION_INFO_V1(prune_using_both_values);
|
PG_FUNCTION_INFO_V1(prune_using_both_values);
|
||||||
PG_FUNCTION_INFO_V1(debug_equality_expression);
|
PG_FUNCTION_INFO_V1(debug_equality_expression);
|
||||||
|
PG_FUNCTION_INFO_V1(print_sorted_shard_intervals);
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
|
@ -140,6 +143,21 @@ debug_equality_expression(PG_FUNCTION_ARGS)
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* print_sorted_shard_intervals prints the sorted shard interval array that is in the
|
||||||
|
* metadata cache. This function aims to test sorting functionality.
|
||||||
|
*/
|
||||||
|
Datum
|
||||||
|
print_sorted_shard_intervals(PG_FUNCTION_ARGS)
|
||||||
|
{
|
||||||
|
Oid distributedTableId = PG_GETARG_OID(0);
|
||||||
|
|
||||||
|
ArrayType *shardIdArrayType = SortedShardIntervalArray(distributedTableId);
|
||||||
|
|
||||||
|
PG_RETURN_ARRAYTYPE_P(shardIdArrayType);
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* MakeTextPartitionExpression returns an equality expression between the
|
* MakeTextPartitionExpression returns an equality expression between the
|
||||||
* specified table's partition column and the provided values.
|
* specified table's partition column and the provided values.
|
||||||
|
@ -212,3 +230,34 @@ PrunedShardIdsForTable(Oid distributedTableId, List *whereClauseList)
|
||||||
|
|
||||||
return shardIdArrayType;
|
return shardIdArrayType;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* SortedShardIntervalArray simply returns the shard interval ids in the sorted shard
|
||||||
|
* interval cache as a datum array.
|
||||||
|
*/
|
||||||
|
static ArrayType *
|
||||||
|
SortedShardIntervalArray(Oid distributedTableId)
|
||||||
|
{
|
||||||
|
ArrayType *shardIdArrayType = NULL;
|
||||||
|
int shardIndex = 0;
|
||||||
|
Oid shardIdTypeId = INT8OID;
|
||||||
|
|
||||||
|
DistTableCacheEntry *cacheEntry = DistributedTableCacheEntry(distributedTableId);
|
||||||
|
ShardInterval **shardIntervalArray = cacheEntry->sortedShardIntervalArray;
|
||||||
|
int shardIdCount = cacheEntry->shardIntervalArrayLength;
|
||||||
|
Datum *shardIdDatumArray = palloc0(shardIdCount * sizeof(Datum));
|
||||||
|
|
||||||
|
for (shardIndex = 0; shardIndex < shardIdCount; ++shardIndex)
|
||||||
|
{
|
||||||
|
ShardInterval *shardId = shardIntervalArray[shardIndex];
|
||||||
|
Datum shardIdDatum = Int64GetDatum(shardId->shardId);
|
||||||
|
|
||||||
|
shardIdDatumArray[shardIndex] = shardIdDatum;
|
||||||
|
}
|
||||||
|
|
||||||
|
shardIdArrayType = DatumArrayToArrayType(shardIdDatumArray, shardIdCount,
|
||||||
|
shardIdTypeId);
|
||||||
|
|
||||||
|
return shardIdArrayType;
|
||||||
|
}
|
||||||
|
|
|
@ -12,6 +12,7 @@
|
||||||
#include "access/genam.h"
|
#include "access/genam.h"
|
||||||
#include "access/heapam.h"
|
#include "access/heapam.h"
|
||||||
#include "access/htup_details.h"
|
#include "access/htup_details.h"
|
||||||
|
#include "access/nbtree.h"
|
||||||
#include "access/xact.h"
|
#include "access/xact.h"
|
||||||
#include "catalog/indexing.h"
|
#include "catalog/indexing.h"
|
||||||
#include "catalog/pg_namespace.h"
|
#include "catalog/pg_namespace.h"
|
||||||
|
@ -22,6 +23,8 @@
|
||||||
#include "distributed/metadata_cache.h"
|
#include "distributed/metadata_cache.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
#include "distributed/pg_dist_shard.h"
|
#include "distributed/pg_dist_shard.h"
|
||||||
|
#include "distributed/shardinterval_utils.h"
|
||||||
|
#include "distributed/worker_protocol.h"
|
||||||
#include "parser/parse_func.h"
|
#include "parser/parse_func.h"
|
||||||
#include "utils/builtins.h"
|
#include "utils/builtins.h"
|
||||||
#include "utils/catcache.h"
|
#include "utils/catcache.h"
|
||||||
|
@ -34,6 +37,7 @@
|
||||||
#include "utils/relfilenodemap.h"
|
#include "utils/relfilenodemap.h"
|
||||||
#include "utils/relmapper.h"
|
#include "utils/relmapper.h"
|
||||||
#include "utils/syscache.h"
|
#include "utils/syscache.h"
|
||||||
|
#include "utils/typcache.h"
|
||||||
|
|
||||||
|
|
||||||
/* state which should be cleared upon DROP EXTENSION */
|
/* state which should be cleared upon DROP EXTENSION */
|
||||||
|
@ -57,6 +61,16 @@ static ScanKeyData DistShardScanKey[1];
|
||||||
|
|
||||||
/* local function forward declarations */
|
/* local function forward declarations */
|
||||||
static DistTableCacheEntry * LookupDistTableCacheEntry(Oid relationId);
|
static DistTableCacheEntry * LookupDistTableCacheEntry(Oid relationId);
|
||||||
|
static FmgrInfo * ShardIntervalCompareFunction(ShardInterval **shardIntervalArray,
|
||||||
|
char partitionMethod);
|
||||||
|
static ShardInterval ** SortShardIntervalArray(ShardInterval **shardIntervalArray,
|
||||||
|
int shardCount,
|
||||||
|
FmgrInfo *
|
||||||
|
shardIntervalSortCompareFunction);
|
||||||
|
static bool HasUniformHashDistribution(ShardInterval **shardIntervalArray,
|
||||||
|
int shardIntervalArrayLength);
|
||||||
|
static bool HasUninitializedShardInterval(ShardInterval **sortedShardIntervalArray,
|
||||||
|
int shardCount);
|
||||||
static void InitializeDistTableCache(void);
|
static void InitializeDistTableCache(void);
|
||||||
static void ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry);
|
static void ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry);
|
||||||
static void InvalidateDistRelationCacheCallback(Datum argument, Oid relationId);
|
static void InvalidateDistRelationCacheCallback(Datum argument, Oid relationId);
|
||||||
|
@ -202,7 +216,12 @@ LookupDistTableCacheEntry(Oid relationId)
|
||||||
char partitionMethod = 0;
|
char partitionMethod = 0;
|
||||||
List *distShardTupleList = NIL;
|
List *distShardTupleList = NIL;
|
||||||
int shardIntervalArrayLength = 0;
|
int shardIntervalArrayLength = 0;
|
||||||
ShardInterval *shardIntervalArray = NULL;
|
ShardInterval **shardIntervalArray = NULL;
|
||||||
|
ShardInterval **sortedShardIntervalArray = NULL;
|
||||||
|
FmgrInfo *shardIntervalCompareFunction = NULL;
|
||||||
|
FmgrInfo *hashFunction = NULL;
|
||||||
|
bool hasUninitializedShardInterval = false;
|
||||||
|
bool hasUniformHashDistribution = false;
|
||||||
void *hashKey = (void *) &relationId;
|
void *hashKey = (void *) &relationId;
|
||||||
|
|
||||||
if (DistTableCacheHash == NULL)
|
if (DistTableCacheHash == NULL)
|
||||||
|
@ -257,7 +276,7 @@ LookupDistTableCacheEntry(Oid relationId)
|
||||||
|
|
||||||
shardIntervalArray = MemoryContextAllocZero(CacheMemoryContext,
|
shardIntervalArray = MemoryContextAllocZero(CacheMemoryContext,
|
||||||
shardIntervalArrayLength *
|
shardIntervalArrayLength *
|
||||||
sizeof(ShardInterval));
|
sizeof(ShardInterval *));
|
||||||
|
|
||||||
foreach(distShardTupleCell, distShardTupleList)
|
foreach(distShardTupleCell, distShardTupleList)
|
||||||
{
|
{
|
||||||
|
@ -266,9 +285,12 @@ LookupDistTableCacheEntry(Oid relationId)
|
||||||
distShardTupleDesc,
|
distShardTupleDesc,
|
||||||
intervalTypeId,
|
intervalTypeId,
|
||||||
intervalTypeMod);
|
intervalTypeMod);
|
||||||
|
ShardInterval *newShardInterval = NULL;
|
||||||
MemoryContext oldContext = MemoryContextSwitchTo(CacheMemoryContext);
|
MemoryContext oldContext = MemoryContextSwitchTo(CacheMemoryContext);
|
||||||
|
|
||||||
CopyShardInterval(shardInterval, &shardIntervalArray[arrayIndex]);
|
newShardInterval = (ShardInterval *) palloc0(sizeof(ShardInterval));
|
||||||
|
CopyShardInterval(shardInterval, newShardInterval);
|
||||||
|
shardIntervalArray[arrayIndex] = newShardInterval;
|
||||||
|
|
||||||
MemoryContextSwitchTo(oldContext);
|
MemoryContextSwitchTo(oldContext);
|
||||||
|
|
||||||
|
@ -280,6 +302,50 @@ LookupDistTableCacheEntry(Oid relationId)
|
||||||
heap_close(distShardRelation, AccessShareLock);
|
heap_close(distShardRelation, AccessShareLock);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/* decide and allocate interval comparison function */
|
||||||
|
if (shardIntervalArrayLength > 0)
|
||||||
|
{
|
||||||
|
MemoryContext oldContext = CurrentMemoryContext;
|
||||||
|
|
||||||
|
/* allocate the comparison function in the cache context */
|
||||||
|
oldContext = MemoryContextSwitchTo(CacheMemoryContext);
|
||||||
|
|
||||||
|
shardIntervalCompareFunction = ShardIntervalCompareFunction(shardIntervalArray,
|
||||||
|
partitionMethod);
|
||||||
|
|
||||||
|
MemoryContextSwitchTo(oldContext);
|
||||||
|
}
|
||||||
|
|
||||||
|
/* sort the interval array */
|
||||||
|
sortedShardIntervalArray = SortShardIntervalArray(shardIntervalArray,
|
||||||
|
shardIntervalArrayLength,
|
||||||
|
shardIntervalCompareFunction);
|
||||||
|
|
||||||
|
/* check if there exists any shard intervals with no min/max values */
|
||||||
|
hasUninitializedShardInterval =
|
||||||
|
HasUninitializedShardInterval(sortedShardIntervalArray, shardIntervalArrayLength);
|
||||||
|
|
||||||
|
/* we only need hash functions for hash distributed tables */
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
TypeCacheEntry *typeEntry = NULL;
|
||||||
|
Node *partitionNode = stringToNode(partitionKeyString);
|
||||||
|
Var *partitionColumn = (Var *) partitionNode;
|
||||||
|
Assert(IsA(partitionNode, Var));
|
||||||
|
typeEntry = lookup_type_cache(partitionColumn->vartype,
|
||||||
|
TYPECACHE_HASH_PROC_FINFO);
|
||||||
|
|
||||||
|
hashFunction = MemoryContextAllocZero(CacheMemoryContext,
|
||||||
|
sizeof(FmgrInfo));
|
||||||
|
|
||||||
|
fmgr_info_copy(hashFunction, &(typeEntry->hash_proc_finfo), CacheMemoryContext);
|
||||||
|
|
||||||
|
/* check the shard distribution for hash partitioned tables */
|
||||||
|
hasUniformHashDistribution =
|
||||||
|
HasUniformHashDistribution(sortedShardIntervalArray,
|
||||||
|
shardIntervalArrayLength);
|
||||||
|
}
|
||||||
|
|
||||||
cacheEntry = hash_search(DistTableCacheHash, hashKey, HASH_ENTER, NULL);
|
cacheEntry = hash_search(DistTableCacheHash, hashKey, HASH_ENTER, NULL);
|
||||||
|
|
||||||
/* zero out entry, but not the key part */
|
/* zero out entry, but not the key part */
|
||||||
|
@ -298,13 +364,151 @@ LookupDistTableCacheEntry(Oid relationId)
|
||||||
cacheEntry->partitionKeyString = partitionKeyString;
|
cacheEntry->partitionKeyString = partitionKeyString;
|
||||||
cacheEntry->partitionMethod = partitionMethod;
|
cacheEntry->partitionMethod = partitionMethod;
|
||||||
cacheEntry->shardIntervalArrayLength = shardIntervalArrayLength;
|
cacheEntry->shardIntervalArrayLength = shardIntervalArrayLength;
|
||||||
cacheEntry->shardIntervalArray = shardIntervalArray;
|
cacheEntry->sortedShardIntervalArray = sortedShardIntervalArray;
|
||||||
|
cacheEntry->shardIntervalCompareFunction = shardIntervalCompareFunction;
|
||||||
|
cacheEntry->hashFunction = hashFunction;
|
||||||
|
cacheEntry->hasUninitializedShardInterval = hasUninitializedShardInterval;
|
||||||
|
cacheEntry->hasUniformHashDistribution = hasUniformHashDistribution;
|
||||||
}
|
}
|
||||||
|
|
||||||
return cacheEntry;
|
return cacheEntry;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* ShardIntervalCompareFunction returns the appropriate compare function for the
|
||||||
|
* partition column type. In case of hash-partitioning, it always returns the compare
|
||||||
|
* function for integers. Callers of this function has to ensure that shardIntervalArray
|
||||||
|
* has at least one element.
|
||||||
|
*/
|
||||||
|
static FmgrInfo *
|
||||||
|
ShardIntervalCompareFunction(ShardInterval **shardIntervalArray, char partitionMethod)
|
||||||
|
{
|
||||||
|
FmgrInfo *shardIntervalCompareFunction = NULL;
|
||||||
|
Oid comparisonTypeId = InvalidOid;
|
||||||
|
|
||||||
|
Assert(shardIntervalArray != NULL);
|
||||||
|
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
comparisonTypeId = INT4OID;
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
ShardInterval *shardInterval = shardIntervalArray[0];
|
||||||
|
comparisonTypeId = shardInterval->valueTypeId;
|
||||||
|
}
|
||||||
|
|
||||||
|
shardIntervalCompareFunction = GetFunctionInfo(comparisonTypeId, BTREE_AM_OID,
|
||||||
|
BTORDER_PROC);
|
||||||
|
|
||||||
|
return shardIntervalCompareFunction;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* SortedShardIntervalArray sorts the input shardIntervalArray. Shard intervals with
|
||||||
|
* no min/max values are placed at the end of the array.
|
||||||
|
*/
|
||||||
|
static ShardInterval **
|
||||||
|
SortShardIntervalArray(ShardInterval **shardIntervalArray, int shardCount,
|
||||||
|
FmgrInfo *shardIntervalSortCompareFunction)
|
||||||
|
{
|
||||||
|
ShardInterval **sortedShardIntervalArray = NULL;
|
||||||
|
|
||||||
|
/* short cut if there are no shard intervals in the array */
|
||||||
|
if (shardCount == 0)
|
||||||
|
{
|
||||||
|
return shardIntervalArray;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* if a shard doesn't have min/max values, it's placed in the end of the array */
|
||||||
|
qsort_arg(shardIntervalArray, shardCount, sizeof(ShardInterval *),
|
||||||
|
(qsort_arg_comparator) CompareShardIntervals,
|
||||||
|
(void *) shardIntervalSortCompareFunction);
|
||||||
|
|
||||||
|
sortedShardIntervalArray = shardIntervalArray;
|
||||||
|
|
||||||
|
return sortedShardIntervalArray;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* HasUniformHashDistribution determines whether the given list of sorted shards
|
||||||
|
* has a uniform hash distribution, as produced by master_create_worker_shards for
|
||||||
|
* hash partitioned tables.
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
HasUniformHashDistribution(ShardInterval **shardIntervalArray,
|
||||||
|
int shardIntervalArrayLength)
|
||||||
|
{
|
||||||
|
uint64 hashTokenIncrement = 0;
|
||||||
|
int shardIndex = 0;
|
||||||
|
|
||||||
|
/* if there are no shards, there is no uniform distribution */
|
||||||
|
if (shardIntervalArrayLength == 0)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* calculate the hash token increment */
|
||||||
|
hashTokenIncrement = HASH_TOKEN_COUNT / shardIntervalArrayLength;
|
||||||
|
|
||||||
|
for (shardIndex = 0; shardIndex < shardIntervalArrayLength; shardIndex++)
|
||||||
|
{
|
||||||
|
ShardInterval *shardInterval = shardIntervalArray[shardIndex];
|
||||||
|
int32 shardMinHashToken = INT32_MIN + (shardIndex * hashTokenIncrement);
|
||||||
|
int32 shardMaxHashToken = shardMinHashToken + (hashTokenIncrement - 1);
|
||||||
|
|
||||||
|
if (shardIndex == (shardIntervalArrayLength - 1))
|
||||||
|
{
|
||||||
|
shardMaxHashToken = INT32_MAX;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (DatumGetInt32(shardInterval->minValue) != shardMinHashToken ||
|
||||||
|
DatumGetInt32(shardInterval->maxValue) != shardMaxHashToken)
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* HasUninitializedShardInterval returns true if all the elements of the
|
||||||
|
* sortedShardIntervalArray has min/max values. Callers of the function must
|
||||||
|
* ensure that input shard interval array is sorted on shardminvalue and uninitialized
|
||||||
|
* shard intervals are at the end of the array.
|
||||||
|
*/
|
||||||
|
static bool
|
||||||
|
HasUninitializedShardInterval(ShardInterval **sortedShardIntervalArray, int shardCount)
|
||||||
|
{
|
||||||
|
bool hasUninitializedShardInterval = false;
|
||||||
|
ShardInterval *lastShardInterval = NULL;
|
||||||
|
|
||||||
|
if (shardCount == 0)
|
||||||
|
{
|
||||||
|
return hasUninitializedShardInterval;
|
||||||
|
}
|
||||||
|
|
||||||
|
Assert(sortedShardIntervalArray != NULL);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Since the shard interval array is sorted, and uninitialized ones stored
|
||||||
|
* in the end of the array, checking the last element is enough.
|
||||||
|
*/
|
||||||
|
lastShardInterval = sortedShardIntervalArray[shardCount - 1];
|
||||||
|
if (!lastShardInterval->minValueExists || !lastShardInterval->maxValueExists)
|
||||||
|
{
|
||||||
|
hasUninitializedShardInterval = true;
|
||||||
|
}
|
||||||
|
|
||||||
|
return hasUninitializedShardInterval;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* CitusHasBeenLoaded returns true if the citus extension has been created
|
* CitusHasBeenLoaded returns true if the citus extension has been created
|
||||||
* in the current database and the extension script has been executed. Otherwise,
|
* in the current database and the extension script has been executed. Otherwise,
|
||||||
|
@ -628,7 +832,7 @@ ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
|
||||||
|
|
||||||
for (i = 0; i < cacheEntry->shardIntervalArrayLength; i++)
|
for (i = 0; i < cacheEntry->shardIntervalArrayLength; i++)
|
||||||
{
|
{
|
||||||
ShardInterval *shardInterval = &cacheEntry->shardIntervalArray[i];
|
ShardInterval *shardInterval = cacheEntry->sortedShardIntervalArray[i];
|
||||||
bool valueByVal = shardInterval->valueByVal;
|
bool valueByVal = shardInterval->valueByVal;
|
||||||
|
|
||||||
if (!valueByVal)
|
if (!valueByVal)
|
||||||
|
@ -643,11 +847,26 @@ ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
|
||||||
pfree(DatumGetPointer(shardInterval->maxValue));
|
pfree(DatumGetPointer(shardInterval->maxValue));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
pfree(shardInterval);
|
||||||
}
|
}
|
||||||
|
|
||||||
pfree(cacheEntry->shardIntervalArray);
|
pfree(cacheEntry->sortedShardIntervalArray);
|
||||||
cacheEntry->shardIntervalArray = NULL;
|
cacheEntry->sortedShardIntervalArray = NULL;
|
||||||
cacheEntry->shardIntervalArrayLength = 0;
|
cacheEntry->shardIntervalArrayLength = 0;
|
||||||
|
|
||||||
|
cacheEntry->hasUninitializedShardInterval = false;
|
||||||
|
cacheEntry->hasUniformHashDistribution = false;
|
||||||
|
|
||||||
|
pfree(cacheEntry->shardIntervalCompareFunction);
|
||||||
|
cacheEntry->shardIntervalCompareFunction = NULL;
|
||||||
|
|
||||||
|
/* we only allocated hash function for hash distributed tables */
|
||||||
|
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
pfree(cacheEntry->hashFunction);
|
||||||
|
cacheEntry->hashFunction = NULL;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,179 @@
|
||||||
|
/*-------------------------------------------------------------------------
|
||||||
|
*
|
||||||
|
* shardinterval_utils.c
|
||||||
|
*
|
||||||
|
* This file contains functions to perform useful operations on shard intervals.
|
||||||
|
*
|
||||||
|
* Copyright (c) 2014-2016, Citus Data, Inc.
|
||||||
|
*
|
||||||
|
*-------------------------------------------------------------------------
|
||||||
|
*/
|
||||||
|
#include "postgres.h"
|
||||||
|
|
||||||
|
#include "access/nbtree.h"
|
||||||
|
#include "catalog/pg_am.h"
|
||||||
|
#include "catalog/pg_collation.h"
|
||||||
|
#include "catalog/pg_type.h"
|
||||||
|
#include "distributed/shardinterval_utils.h"
|
||||||
|
#include "distributed/pg_dist_partition.h"
|
||||||
|
#include "distributed/worker_protocol.h"
|
||||||
|
#include "utils/catcache.h"
|
||||||
|
#include "utils/memutils.h"
|
||||||
|
|
||||||
|
|
||||||
|
static ShardInterval * SearchCachedShardInterval(Datum partitionColumnValue,
|
||||||
|
ShardInterval **shardIntervalCache,
|
||||||
|
int shardCount,
|
||||||
|
FmgrInfo *compareFunction);
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* CompareShardIntervals acts as a helper function to compare two shard intervals
|
||||||
|
* by their minimum values, using the value's type comparison function.
|
||||||
|
*
|
||||||
|
* If a shard interval does not have min/max value, it's treated as being greater
|
||||||
|
* than the other.
|
||||||
|
*/
|
||||||
|
int
|
||||||
|
CompareShardIntervals(const void *leftElement, const void *rightElement,
|
||||||
|
FmgrInfo *typeCompareFunction)
|
||||||
|
{
|
||||||
|
ShardInterval *leftShardInterval = *((ShardInterval **) leftElement);
|
||||||
|
ShardInterval *rightShardInterval = *((ShardInterval **) rightElement);
|
||||||
|
Datum leftDatum = 0;
|
||||||
|
Datum rightDatum = 0;
|
||||||
|
Datum comparisonDatum = 0;
|
||||||
|
int comparisonResult = 0;
|
||||||
|
|
||||||
|
Assert(typeCompareFunction != NULL);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Left element should be treated as the greater element in case it doesn't
|
||||||
|
* have min or max values.
|
||||||
|
*/
|
||||||
|
if (!leftShardInterval->minValueExists || !leftShardInterval->maxValueExists)
|
||||||
|
{
|
||||||
|
comparisonResult = 1;
|
||||||
|
return comparisonResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Right element should be treated as the greater element in case it doesn't
|
||||||
|
* have min or max values.
|
||||||
|
*/
|
||||||
|
if (!rightShardInterval->minValueExists || !rightShardInterval->maxValueExists)
|
||||||
|
{
|
||||||
|
comparisonResult = -1;
|
||||||
|
return comparisonResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
/* if both shard interval have min/max values, calculate the comparison result */
|
||||||
|
leftDatum = leftShardInterval->minValue;
|
||||||
|
rightDatum = rightShardInterval->minValue;
|
||||||
|
|
||||||
|
comparisonDatum = CompareCall2(typeCompareFunction, leftDatum, rightDatum);
|
||||||
|
comparisonResult = DatumGetInt32(comparisonDatum);
|
||||||
|
|
||||||
|
return comparisonResult;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* FindShardInterval finds a single shard interval in the cache for the
|
||||||
|
* given partition column value.
|
||||||
|
*/
|
||||||
|
ShardInterval *
|
||||||
|
FindShardInterval(Datum partitionColumnValue, ShardInterval **shardIntervalCache,
|
||||||
|
int shardCount, char partitionMethod, FmgrInfo *compareFunction,
|
||||||
|
FmgrInfo *hashFunction, bool useBinarySearch)
|
||||||
|
{
|
||||||
|
ShardInterval *shardInterval = NULL;
|
||||||
|
|
||||||
|
if (partitionMethod == DISTRIBUTE_BY_HASH)
|
||||||
|
{
|
||||||
|
int hashedValue = DatumGetInt32(FunctionCall1(hashFunction,
|
||||||
|
partitionColumnValue));
|
||||||
|
if (useBinarySearch)
|
||||||
|
{
|
||||||
|
Assert(compareFunction != NULL);
|
||||||
|
|
||||||
|
shardInterval = SearchCachedShardInterval(Int32GetDatum(hashedValue),
|
||||||
|
shardIntervalCache, shardCount,
|
||||||
|
compareFunction);
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
uint64 hashTokenIncrement = HASH_TOKEN_COUNT / shardCount;
|
||||||
|
int shardIndex = (uint32) (hashedValue - INT32_MIN) / hashTokenIncrement;
|
||||||
|
|
||||||
|
Assert(shardIndex <= shardCount);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* If the shard count is not power of 2, the range of the last
|
||||||
|
* shard becomes larger than others. For that extra piece of range,
|
||||||
|
* we still need to use the last shard.
|
||||||
|
*/
|
||||||
|
if (shardIndex == shardCount)
|
||||||
|
{
|
||||||
|
shardIndex = shardCount - 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
shardInterval = shardIntervalCache[shardIndex];
|
||||||
|
}
|
||||||
|
}
|
||||||
|
else
|
||||||
|
{
|
||||||
|
Assert(compareFunction != NULL);
|
||||||
|
|
||||||
|
shardInterval = SearchCachedShardInterval(partitionColumnValue,
|
||||||
|
shardIntervalCache, shardCount,
|
||||||
|
compareFunction);
|
||||||
|
}
|
||||||
|
|
||||||
|
return shardInterval;
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
/*
|
||||||
|
* SearchCachedShardInterval performs a binary search for a shard interval matching a
|
||||||
|
* given partition column value and returns it.
|
||||||
|
*/
|
||||||
|
static ShardInterval *
|
||||||
|
SearchCachedShardInterval(Datum partitionColumnValue, ShardInterval **shardIntervalCache,
|
||||||
|
int shardCount, FmgrInfo *compareFunction)
|
||||||
|
{
|
||||||
|
int lowerBoundIndex = 0;
|
||||||
|
int upperBoundIndex = shardCount;
|
||||||
|
|
||||||
|
while (lowerBoundIndex < upperBoundIndex)
|
||||||
|
{
|
||||||
|
int middleIndex = (lowerBoundIndex + upperBoundIndex) / 2;
|
||||||
|
int maxValueComparison = 0;
|
||||||
|
int minValueComparison = 0;
|
||||||
|
|
||||||
|
minValueComparison = FunctionCall2Coll(compareFunction,
|
||||||
|
DEFAULT_COLLATION_OID,
|
||||||
|
partitionColumnValue,
|
||||||
|
shardIntervalCache[middleIndex]->minValue);
|
||||||
|
|
||||||
|
if (DatumGetInt32(minValueComparison) < 0)
|
||||||
|
{
|
||||||
|
upperBoundIndex = middleIndex;
|
||||||
|
continue;
|
||||||
|
}
|
||||||
|
|
||||||
|
maxValueComparison = FunctionCall2Coll(compareFunction,
|
||||||
|
DEFAULT_COLLATION_OID,
|
||||||
|
partitionColumnValue,
|
||||||
|
shardIntervalCache[middleIndex]->maxValue);
|
||||||
|
|
||||||
|
if (DatumGetInt32(maxValueComparison) <= 0)
|
||||||
|
{
|
||||||
|
return shardIntervalCache[middleIndex];
|
||||||
|
}
|
||||||
|
|
||||||
|
lowerBoundIndex = middleIndex + 1;
|
||||||
|
}
|
||||||
|
|
||||||
|
return NULL;
|
||||||
|
}
|
|
@ -11,6 +11,7 @@
|
||||||
#ifndef METADATA_CACHE_H
|
#ifndef METADATA_CACHE_H
|
||||||
#define METADATA_CACHE_H
|
#define METADATA_CACHE_H
|
||||||
|
|
||||||
|
#include "fmgr.h"
|
||||||
#include "distributed/master_metadata_utility.h"
|
#include "distributed/master_metadata_utility.h"
|
||||||
#include "distributed/pg_dist_partition.h"
|
#include "distributed/pg_dist_partition.h"
|
||||||
|
|
||||||
|
@ -31,6 +32,8 @@ typedef struct
|
||||||
bool isValid;
|
bool isValid;
|
||||||
|
|
||||||
bool isDistributedTable;
|
bool isDistributedTable;
|
||||||
|
bool hasUninitializedShardInterval;
|
||||||
|
bool hasUniformHashDistribution; /* valid for hash partitioned tables */
|
||||||
|
|
||||||
/* pg_dist_partition metadata for this table */
|
/* pg_dist_partition metadata for this table */
|
||||||
char *partitionKeyString;
|
char *partitionKeyString;
|
||||||
|
@ -38,7 +41,10 @@ typedef struct
|
||||||
|
|
||||||
/* pg_dist_shard metadata (variable-length ShardInterval array) for this table */
|
/* pg_dist_shard metadata (variable-length ShardInterval array) for this table */
|
||||||
int shardIntervalArrayLength;
|
int shardIntervalArrayLength;
|
||||||
ShardInterval *shardIntervalArray;
|
ShardInterval **sortedShardIntervalArray;
|
||||||
|
|
||||||
|
FmgrInfo *shardIntervalCompareFunction; /* NULL if no shard intervals exist */
|
||||||
|
FmgrInfo *hashFunction; /* NULL if table is not distributed by hash */
|
||||||
} DistTableCacheEntry;
|
} DistTableCacheEntry;
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -237,7 +237,6 @@ extern bool OpExpressionContainsColumn(OpExpr *operatorExpression, Var *partitio
|
||||||
extern int CompareShardPlacements(const void *leftElement, const void *rightElement);
|
extern int CompareShardPlacements(const void *leftElement, const void *rightElement);
|
||||||
|
|
||||||
/* Function declarations for sorting shards. */
|
/* Function declarations for sorting shards. */
|
||||||
extern ShardInterval ** SortedShardIntervalArray(List *shardList);
|
|
||||||
extern bool ShardIntervalsOverlap(ShardInterval *firstInterval,
|
extern bool ShardIntervalsOverlap(ShardInterval *firstInterval,
|
||||||
ShardInterval *secondInterval);
|
ShardInterval *secondInterval);
|
||||||
|
|
||||||
|
|
|
@ -0,0 +1,35 @@
|
||||||
|
/*-------------------------------------------------------------------------
|
||||||
|
*
|
||||||
|
* shardinterval_utils.h
|
||||||
|
*
|
||||||
|
* Declarations for public utility functions related to shard intervals.
|
||||||
|
*
|
||||||
|
* Copyright (c) 2014-2016, Citus Data, Inc.
|
||||||
|
*
|
||||||
|
*-------------------------------------------------------------------------
|
||||||
|
*/
|
||||||
|
|
||||||
|
#ifndef SHARDINTERVAL_UTILS_H_
|
||||||
|
#define SHARDINTERVAL_UTILS_H_
|
||||||
|
|
||||||
|
#include "distributed/master_metadata_utility.h"
|
||||||
|
#include "nodes/primnodes.h"
|
||||||
|
|
||||||
|
/* OperatorCacheEntry contains information for each element in OperatorCache */
|
||||||
|
typedef struct ShardIntervalCompareFunctionCacheEntry
|
||||||
|
{
|
||||||
|
Var *partitionColumn;
|
||||||
|
char partitionMethod;
|
||||||
|
FmgrInfo *functionInfo;
|
||||||
|
} ShardIntervalCompareFunctionCacheEntry;
|
||||||
|
|
||||||
|
extern int CompareShardIntervals(const void *leftElement, const void *rightElement,
|
||||||
|
FmgrInfo *typeCompareFunction);
|
||||||
|
extern ShardInterval * FindShardInterval(Datum partitionColumnValue,
|
||||||
|
ShardInterval **shardIntervalCache,
|
||||||
|
int shardCount, char partitionMethod,
|
||||||
|
FmgrInfo *compareFunction,
|
||||||
|
FmgrInfo *hashFunction, bool useBinarySearch);
|
||||||
|
|
||||||
|
|
||||||
|
#endif /* SHARDINTERVAL_UTILS_H_ */
|
|
@ -87,7 +87,7 @@ VALUES
|
||||||
SELECT load_shard_id_array('events_hash');
|
SELECT load_shard_id_array('events_hash');
|
||||||
load_shard_id_array
|
load_shard_id_array
|
||||||
---------------------
|
---------------------
|
||||||
{4,3,2,1}
|
{1,2,3,4}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- should see array with first shard range
|
-- should see array with first shard range
|
||||||
|
|
|
@ -73,25 +73,25 @@ DEBUG: predicate pruning for shardId 111
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 2;
|
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 2;
|
||||||
DEBUG: predicate pruning for shardId 113
|
|
||||||
DEBUG: predicate pruning for shardId 112
|
DEBUG: predicate pruning for shardId 112
|
||||||
DEBUG: predicate pruning for shardId 110
|
DEBUG: predicate pruning for shardId 110
|
||||||
|
DEBUG: predicate pruning for shardId 113
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 3;
|
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 3;
|
||||||
DEBUG: predicate pruning for shardId 111
|
|
||||||
DEBUG: predicate pruning for shardId 110
|
DEBUG: predicate pruning for shardId 110
|
||||||
|
DEBUG: predicate pruning for shardId 111
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 4;
|
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey = 4;
|
||||||
DEBUG: predicate pruning for shardId 111
|
|
||||||
DEBUG: predicate pruning for shardId 110
|
DEBUG: predicate pruning for shardId 110
|
||||||
|
DEBUG: predicate pruning for shardId 111
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
@ -99,8 +99,8 @@ DEBUG: predicate pruning for shardId 110
|
||||||
|
|
||||||
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey is NULL;
|
EXPLAIN SELECT count(*) FROM orders_hash_partitioned WHERE o_orderkey is NULL;
|
||||||
DEBUG: predicate pruning for shardId 112
|
DEBUG: predicate pruning for shardId 112
|
||||||
DEBUG: predicate pruning for shardId 111
|
|
||||||
DEBUG: predicate pruning for shardId 110
|
DEBUG: predicate pruning for shardId 110
|
||||||
|
DEBUG: predicate pruning for shardId 111
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
@ -231,14 +231,14 @@ DEBUG: predicate pruning for shardId 111
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN SELECT count(*)
|
||||||
FROM orders_hash_partitioned orders1, orders_hash_partitioned orders2
|
FROM orders_hash_partitioned orders1, orders_hash_partitioned orders2
|
||||||
WHERE orders1.o_orderkey = orders2.o_orderkey;
|
WHERE orders1.o_orderkey = orders2.o_orderkey;
|
||||||
DEBUG: join prunable for intervals [-1011077333,0] and [1134484726,1134484726]
|
|
||||||
DEBUG: join prunable for intervals [-1011077333,0] and [-1905060026,-1905060026]
|
|
||||||
DEBUG: join prunable for intervals [-1905060026,-28094569] and [1134484726,1134484726]
|
DEBUG: join prunable for intervals [-1905060026,-28094569] and [1134484726,1134484726]
|
||||||
DEBUG: join prunable for intervals [1134484726,1134484726] and [-1011077333,0]
|
|
||||||
DEBUG: join prunable for intervals [1134484726,1134484726] and [-1905060026,-28094569]
|
|
||||||
DEBUG: join prunable for intervals [1134484726,1134484726] and [-1905060026,-1905060026]
|
|
||||||
DEBUG: join prunable for intervals [-1905060026,-1905060026] and [-1011077333,0]
|
DEBUG: join prunable for intervals [-1905060026,-1905060026] and [-1011077333,0]
|
||||||
DEBUG: join prunable for intervals [-1905060026,-1905060026] and [1134484726,1134484726]
|
DEBUG: join prunable for intervals [-1905060026,-1905060026] and [1134484726,1134484726]
|
||||||
|
DEBUG: join prunable for intervals [-1011077333,0] and [-1905060026,-1905060026]
|
||||||
|
DEBUG: join prunable for intervals [-1011077333,0] and [1134484726,1134484726]
|
||||||
|
DEBUG: join prunable for intervals [1134484726,1134484726] and [-1905060026,-28094569]
|
||||||
|
DEBUG: join prunable for intervals [1134484726,1134484726] and [-1905060026,-1905060026]
|
||||||
|
DEBUG: join prunable for intervals [1134484726,1134484726] and [-1011077333,0]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
@ -252,8 +252,8 @@ EXPLAIN SELECT count(*)
|
||||||
DEBUG: predicate pruning for shardId 113
|
DEBUG: predicate pruning for shardId 113
|
||||||
DEBUG: predicate pruning for shardId 111
|
DEBUG: predicate pruning for shardId 111
|
||||||
DEBUG: predicate pruning for shardId 112
|
DEBUG: predicate pruning for shardId 112
|
||||||
DEBUG: predicate pruning for shardId 111
|
|
||||||
DEBUG: predicate pruning for shardId 110
|
DEBUG: predicate pruning for shardId 110
|
||||||
|
DEBUG: predicate pruning for shardId 111
|
||||||
DEBUG: join prunable for intervals [-1905060026,-1905060026] and [-1011077333,0]
|
DEBUG: join prunable for intervals [-1905060026,-1905060026] and [-1011077333,0]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
|
|
|
@ -137,11 +137,11 @@ ERROR: creating unique indexes on append-partitioned tables is currently unsupp
|
||||||
CREATE INDEX lineitem_orderkey_index ON lineitem (l_orderkey);
|
CREATE INDEX lineitem_orderkey_index ON lineitem (l_orderkey);
|
||||||
ERROR: relation "lineitem_orderkey_index" already exists
|
ERROR: relation "lineitem_orderkey_index" already exists
|
||||||
CREATE INDEX try_index ON lineitem USING gist (l_orderkey);
|
CREATE INDEX try_index ON lineitem USING gist (l_orderkey);
|
||||||
WARNING: could not receive query results from localhost:57638
|
WARNING: could not receive query results from localhost:57637
|
||||||
DETAIL: Client error: data type bigint has no default operator class for access method "gist"
|
DETAIL: Client error: data type bigint has no default operator class for access method "gist"
|
||||||
ERROR: could not execute DDL command on worker node shards
|
ERROR: could not execute DDL command on worker node shards
|
||||||
CREATE INDEX try_index ON lineitem (non_existent_column);
|
CREATE INDEX try_index ON lineitem (non_existent_column);
|
||||||
WARNING: could not receive query results from localhost:57638
|
WARNING: could not receive query results from localhost:57637
|
||||||
DETAIL: Client error: column "non_existent_column" does not exist
|
DETAIL: Client error: column "non_existent_column" does not exist
|
||||||
ERROR: could not execute DDL command on worker node shards
|
ERROR: could not execute DDL command on worker node shards
|
||||||
CREATE INDEX ON lineitem (l_orderkey);
|
CREATE INDEX ON lineitem (l_orderkey);
|
||||||
|
|
|
@ -8,34 +8,34 @@ SET client_min_messages TO DEBUG2;
|
||||||
EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2
|
EXPLAIN SELECT l1.l_quantity FROM lineitem l1, lineitem l2
|
||||||
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
|
WHERE l1.l_orderkey = l2.l_orderkey AND l1.l_quantity > 5;
|
||||||
LOG: join order: [ "lineitem" ][ local partition join "lineitem" ]
|
LOG: join order: [ "lineitem" ][ local partition join "lineitem" ]
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [11554,13920]
|
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [8997,11554]
|
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [4965,5986]
|
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [2497,4964]
|
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,2496]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [13921,14947]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [4965,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [2497,4964]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,2496]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [13921,14947]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [4965,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [2497,4964]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,2496]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [13921,14947]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [11554,13920]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,11554]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [2497,4964]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [1,2496]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [13921,14947]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [11554,13920]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,11554]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [4965,5986]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [1,2496]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [13921,14947]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [11554,13920]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [8997,11554]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [4965,5986]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [2497,4964]
|
DEBUG: join prunable for intervals [1,2496] and [2497,4964]
|
||||||
|
DEBUG: join prunable for intervals [1,2496] and [4965,5986]
|
||||||
|
DEBUG: join prunable for intervals [1,2496] and [8997,11554]
|
||||||
|
DEBUG: join prunable for intervals [1,2496] and [11554,13920]
|
||||||
|
DEBUG: join prunable for intervals [1,2496] and [13921,14947]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [1,2496]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [4965,5986]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [8997,11554]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [11554,13920]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [13921,14947]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [1,2496]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [2497,4964]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,11554]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [11554,13920]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [13921,14947]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,2496]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [2497,4964]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [4965,5986]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [13921,14947]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,2496]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [2497,4964]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [4965,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [13921,14947]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,2496]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [2497,4964]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [4965,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [8997,11554]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [11554,13920]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
|
|
@ -9,12 +9,12 @@ SET client_min_messages TO DEBUG2;
|
||||||
SET citus.large_table_shard_count TO 2;
|
SET citus.large_table_shard_count TO 2;
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey;
|
WHERE l_orderkey = o_orderkey;
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
36086 | 3.0076679446574429
|
36086 | 3.0076679446574429
|
||||||
|
@ -22,12 +22,12 @@ DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
||||||
|
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey AND l_orderkey > 9030;
|
WHERE l_orderkey = o_orderkey AND l_orderkey > 9030;
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
DEBUG: predicate pruning for shardId 102010
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
DEBUG: predicate pruning for shardId 102011
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
17996 | 3.0194630872483221
|
17996 | 3.0194630872483221
|
||||||
|
@ -37,12 +37,12 @@ DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
-- works as expected in this case.
|
-- works as expected in this case.
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey AND l_orderkey > 20000;
|
WHERE l_orderkey = o_orderkey AND l_orderkey > 20000;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102012
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102012
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
sum | avg
|
sum | avg
|
||||||
-----+-----
|
-----+-----
|
||||||
|
|
|
|
||||||
|
@ -53,13 +53,13 @@ DEBUG: predicate pruning for shardId 102009
|
||||||
-- out all the shards, and leave us with an empty task list.
|
-- out all the shards, and leave us with an empty task list.
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey AND l_orderkey > 6000 AND o_orderkey < 6000;
|
WHERE l_orderkey = o_orderkey AND l_orderkey > 6000 AND o_orderkey < 6000;
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
DEBUG: predicate pruning for shardId 102016
|
DEBUG: predicate pruning for shardId 102016
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-----+-----
|
-----+-----
|
||||||
|
|
|
|
||||||
|
@ -72,8 +72,8 @@ DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN SELECT count(*)
|
||||||
FROM array_partitioned_table table1, array_partitioned_table table2
|
FROM array_partitioned_table table1, array_partitioned_table table2
|
||||||
WHERE table1.array_column = table2.array_column;
|
WHERE table1.array_column = table2.array_column;
|
||||||
DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}]
|
|
||||||
DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}]
|
DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}]
|
||||||
|
DEBUG: join prunable for intervals [{BA1000U2AMO4ZGX,BZZXSP27F21T6},{CA1000U2AMO4ZGX,CZZXSP27F21T6}] and [{},{AZZXSP27F21T6,AZZXSP27F21T6}]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
@ -82,8 +82,8 @@ DEBUG: join prunable for intervals [{},{AZZXSP27F21T6,AZZXSP27F21T6}] and [{BA1
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN SELECT count(*)
|
||||||
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
FROM composite_partitioned_table table1, composite_partitioned_table table2
|
||||||
WHERE table1.composite_column = table2.composite_column;
|
WHERE table1.composite_column = table2.composite_column;
|
||||||
DEBUG: join prunable for intervals [(c,5,d),(d,6,e)] and [(a,3,b),(b,4,c)]
|
|
||||||
DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)]
|
DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)]
|
||||||
|
DEBUG: join prunable for intervals [(c,5,d),(d,6,e)] and [(a,3,b),(b,4,c)]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
@ -93,8 +93,8 @@ DEBUG: join prunable for intervals [(a,3,b),(b,4,c)] and [(c,5,d),(d,6,e)]
|
||||||
EXPLAIN SELECT count(*)
|
EXPLAIN SELECT count(*)
|
||||||
FROM varchar_partitioned_table table1, varchar_partitioned_table table2
|
FROM varchar_partitioned_table table1, varchar_partitioned_table table2
|
||||||
WHERE table1.varchar_column = table2.varchar_column;
|
WHERE table1.varchar_column = table2.varchar_column;
|
||||||
DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6]
|
|
||||||
DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U2AMO4ZGX,BZZXSP27F21T6]
|
DEBUG: join prunable for intervals [AA1000U2AMO4ZGX,AZZXSP27F21T6] and [BA1000U2AMO4ZGX,BZZXSP27F21T6]
|
||||||
|
DEBUG: join prunable for intervals [BA1000U2AMO4ZGX,BZZXSP27F21T6] and [AA1000U2AMO4ZGX,AZZXSP27F21T6]
|
||||||
QUERY PLAN
|
QUERY PLAN
|
||||||
----------------------------------------------------------------------
|
----------------------------------------------------------------------
|
||||||
explain statements for distributed queries are currently unsupported
|
explain statements for distributed queries are currently unsupported
|
||||||
|
|
|
@ -45,30 +45,30 @@ ORDER BY
|
||||||
LIMIT 30;
|
LIMIT 30;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: push down of limit count: 30
|
DEBUG: push down of limit count: 30
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
DEBUG: generated sql query for job 1250 and task 3
|
DEBUG: generated sql query for job 1250 and task 3
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102014 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 6
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102013 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 9
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102012 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 12
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102011 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 15
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102010 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 18
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102009 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102009 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
||||||
DEBUG: assigned task 15 to node localhost:57637
|
DEBUG: generated sql query for job 1250 and task 6
|
||||||
DEBUG: assigned task 18 to node localhost:57638
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102010 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
||||||
DEBUG: assigned task 9 to node localhost:57637
|
DEBUG: generated sql query for job 1250 and task 9
|
||||||
DEBUG: assigned task 12 to node localhost:57638
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102011 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
||||||
DEBUG: assigned task 3 to node localhost:57637
|
DEBUG: generated sql query for job 1250 and task 12
|
||||||
DEBUG: assigned task 6 to node localhost:57638
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102012 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
||||||
|
DEBUG: generated sql query for job 1250 and task 15
|
||||||
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102013 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
||||||
|
DEBUG: generated sql query for job 1250 and task 18
|
||||||
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102014 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > '10'::numeric)"
|
||||||
|
DEBUG: assigned task 6 to node localhost:57637
|
||||||
|
DEBUG: assigned task 3 to node localhost:57638
|
||||||
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
|
DEBUG: assigned task 9 to node localhost:57638
|
||||||
|
DEBUG: assigned task 18 to node localhost:57637
|
||||||
|
DEBUG: assigned task 15 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
||||||
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
|
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
|
||||||
DEBUG: generated sql query for job 1251 and task 3
|
DEBUG: generated sql query for job 1251 and task 3
|
||||||
|
@ -83,10 +83,10 @@ DEBUG: assigned task 3 to node localhost:57637
|
||||||
DEBUG: assigned task 6 to node localhost:57638
|
DEBUG: assigned task 6 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
||||||
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [6001,7000]
|
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
||||||
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
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 [1,1000]
|
||||||
|
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
||||||
DEBUG: generated sql query for job 1252 and task 3
|
DEBUG: generated sql query for job 1252 and task 3
|
||||||
DETAIL: query string: "SELECT "pg_merge_job_1251.task_000007".intermediate_column_1251_0 AS l_partkey, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_1251.task_000007 "pg_merge_job_1251.task_000007" JOIN customer_102017 customer ON ((customer.c_custkey = "pg_merge_job_1251.task_000007".intermediate_column_1251_4))) WHERE ((("pg_merge_job_1251.task_000007".intermediate_column_1251_2 > 5.0) OR ("pg_merge_job_1251.task_000007".intermediate_column_1251_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 ORDER BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1, "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 LIMIT '30'::bigint"
|
DETAIL: query string: "SELECT "pg_merge_job_1251.task_000007".intermediate_column_1251_0 AS l_partkey, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_1251.task_000007 "pg_merge_job_1251.task_000007" JOIN customer_102017 customer ON ((customer.c_custkey = "pg_merge_job_1251.task_000007".intermediate_column_1251_4))) WHERE ((("pg_merge_job_1251.task_000007".intermediate_column_1251_2 > 5.0) OR ("pg_merge_job_1251.task_000007".intermediate_column_1251_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 ORDER BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1, "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 LIMIT '30'::bigint"
|
||||||
DEBUG: generated sql query for job 1252 and task 6
|
DEBUG: generated sql query for job 1252 and task 6
|
||||||
|
@ -156,29 +156,29 @@ ORDER BY
|
||||||
l_partkey, o_orderkey;
|
l_partkey, o_orderkey;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: generated sql query for job 1253 and task 2
|
DEBUG: generated sql query for job 1253 and task 2
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102014 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 4
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102013 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 6
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102012 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 8
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102011 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 10
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102010 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 12
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102009 lineitem WHERE (l_quantity < 5.0)"
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102009 lineitem WHERE (l_quantity < 5.0)"
|
||||||
DEBUG: assigned task 10 to node localhost:57637
|
DEBUG: generated sql query for job 1253 and task 4
|
||||||
DEBUG: assigned task 12 to node localhost:57638
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102010 lineitem WHERE (l_quantity < 5.0)"
|
||||||
DEBUG: assigned task 6 to node localhost:57637
|
DEBUG: generated sql query for job 1253 and task 6
|
||||||
DEBUG: assigned task 8 to node localhost:57638
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102011 lineitem WHERE (l_quantity < 5.0)"
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
DEBUG: generated sql query for job 1253 and task 8
|
||||||
DEBUG: assigned task 4 to node localhost:57638
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102012 lineitem WHERE (l_quantity < 5.0)"
|
||||||
|
DEBUG: generated sql query for job 1253 and task 10
|
||||||
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102013 lineitem WHERE (l_quantity < 5.0)"
|
||||||
|
DEBUG: generated sql query for job 1253 and task 12
|
||||||
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102014 lineitem WHERE (l_quantity < 5.0)"
|
||||||
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
|
DEBUG: assigned task 8 to node localhost:57637
|
||||||
|
DEBUG: assigned task 6 to node localhost:57638
|
||||||
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
|
DEBUG: assigned task 10 to node localhost:57638
|
||||||
DEBUG: generated sql query for job 1254 and task 2
|
DEBUG: generated sql query for job 1254 and task 2
|
||||||
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102016 orders WHERE (o_totalprice <> 4.0)"
|
|
||||||
DEBUG: generated sql query for job 1254 and task 4
|
|
||||||
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102015 orders WHERE (o_totalprice <> 4.0)"
|
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102015 orders WHERE (o_totalprice <> 4.0)"
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
DEBUG: generated sql query for job 1254 and task 4
|
||||||
DEBUG: assigned task 4 to node localhost:57638
|
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102016 orders WHERE (o_totalprice <> 4.0)"
|
||||||
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
DEBUG: join prunable for task partitionId 0 and 1
|
DEBUG: join prunable for task partitionId 0 and 1
|
||||||
DEBUG: join prunable for task partitionId 0 and 2
|
DEBUG: join prunable for task partitionId 0 and 2
|
||||||
DEBUG: join prunable for task partitionId 0 and 3
|
DEBUG: join prunable for task partitionId 0 and 3
|
||||||
|
|
|
@ -45,30 +45,30 @@ ORDER BY
|
||||||
LIMIT 30;
|
LIMIT 30;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: push down of limit count: 30
|
DEBUG: push down of limit count: 30
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
DEBUG: generated sql query for job 1250 and task 3
|
DEBUG: generated sql query for job 1250 and task 3
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102014 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 6
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102013 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 9
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102012 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 12
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102011 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 15
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102010 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
|
||||||
DEBUG: generated sql query for job 1250 and task 18
|
|
||||||
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102009 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102009 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
||||||
DEBUG: assigned task 15 to node localhost:57637
|
DEBUG: generated sql query for job 1250 and task 6
|
||||||
DEBUG: assigned task 18 to node localhost:57638
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102010 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
||||||
DEBUG: assigned task 9 to node localhost:57637
|
DEBUG: generated sql query for job 1250 and task 9
|
||||||
DEBUG: assigned task 12 to node localhost:57638
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102011 lineitem JOIN orders_102015 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
||||||
DEBUG: assigned task 3 to node localhost:57637
|
DEBUG: generated sql query for job 1250 and task 12
|
||||||
DEBUG: assigned task 6 to node localhost:57638
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102012 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
||||||
|
DEBUG: generated sql query for job 1250 and task 15
|
||||||
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102013 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
||||||
|
DEBUG: generated sql query for job 1250 and task 18
|
||||||
|
DETAIL: query string: "SELECT lineitem.l_partkey, orders.o_orderkey, lineitem.l_quantity, lineitem.l_extendedprice, orders.o_custkey FROM (lineitem_102014 lineitem JOIN orders_102016 orders ON ((lineitem.l_orderkey = orders.o_orderkey))) WHERE (orders.o_totalprice > 10::numeric)"
|
||||||
|
DEBUG: assigned task 6 to node localhost:57637
|
||||||
|
DEBUG: assigned task 3 to node localhost:57638
|
||||||
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
|
DEBUG: assigned task 9 to node localhost:57638
|
||||||
|
DEBUG: assigned task 18 to node localhost:57637
|
||||||
|
DEBUG: assigned task 15 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
||||||
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
|
DEBUG: join prunable for intervals [6001,7000] and [1,1000]
|
||||||
DEBUG: generated sql query for job 1251 and task 3
|
DEBUG: generated sql query for job 1251 and task 3
|
||||||
|
@ -83,10 +83,10 @@ DEBUG: assigned task 3 to node localhost:57637
|
||||||
DEBUG: assigned task 6 to node localhost:57638
|
DEBUG: assigned task 6 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
||||||
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [6001,7000]
|
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
||||||
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
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 [1,1000]
|
||||||
|
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
||||||
DEBUG: generated sql query for job 1252 and task 3
|
DEBUG: generated sql query for job 1252 and task 3
|
||||||
DETAIL: query string: "SELECT "pg_merge_job_1251.task_000007".intermediate_column_1251_0 AS l_partkey, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_1251.task_000007 "pg_merge_job_1251.task_000007" JOIN customer_102017 customer ON ((customer.c_custkey = "pg_merge_job_1251.task_000007".intermediate_column_1251_4))) WHERE ((("pg_merge_job_1251.task_000007".intermediate_column_1251_2 > 5.0) OR ("pg_merge_job_1251.task_000007".intermediate_column_1251_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 ORDER BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1, "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 LIMIT 30::bigint"
|
DETAIL: query string: "SELECT "pg_merge_job_1251.task_000007".intermediate_column_1251_0 AS l_partkey, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 AS o_orderkey, count(*) AS count FROM (pg_merge_job_1251.task_000007 "pg_merge_job_1251.task_000007" JOIN customer_102017 customer ON ((customer.c_custkey = "pg_merge_job_1251.task_000007".intermediate_column_1251_4))) WHERE ((("pg_merge_job_1251.task_000007".intermediate_column_1251_2 > 5.0) OR ("pg_merge_job_1251.task_000007".intermediate_column_1251_3 > 1200.0)) AND (customer.c_acctbal < 5000.0)) GROUP BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 ORDER BY "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1, "pg_merge_job_1251.task_000007".intermediate_column_1251_0, "pg_merge_job_1251.task_000007".intermediate_column_1251_1 LIMIT 30::bigint"
|
||||||
DEBUG: generated sql query for job 1252 and task 6
|
DEBUG: generated sql query for job 1252 and task 6
|
||||||
|
@ -156,29 +156,29 @@ ORDER BY
|
||||||
l_partkey, o_orderkey;
|
l_partkey, o_orderkey;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: generated sql query for job 1253 and task 2
|
DEBUG: generated sql query for job 1253 and task 2
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102014 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 4
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102013 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 6
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102012 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 8
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102011 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 10
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102010 lineitem WHERE (l_quantity < 5.0)"
|
|
||||||
DEBUG: generated sql query for job 1253 and task 12
|
|
||||||
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102009 lineitem WHERE (l_quantity < 5.0)"
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102009 lineitem WHERE (l_quantity < 5.0)"
|
||||||
DEBUG: assigned task 10 to node localhost:57637
|
DEBUG: generated sql query for job 1253 and task 4
|
||||||
DEBUG: assigned task 12 to node localhost:57638
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102010 lineitem WHERE (l_quantity < 5.0)"
|
||||||
DEBUG: assigned task 6 to node localhost:57637
|
DEBUG: generated sql query for job 1253 and task 6
|
||||||
DEBUG: assigned task 8 to node localhost:57638
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102011 lineitem WHERE (l_quantity < 5.0)"
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
DEBUG: generated sql query for job 1253 and task 8
|
||||||
DEBUG: assigned task 4 to node localhost:57638
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102012 lineitem WHERE (l_quantity < 5.0)"
|
||||||
|
DEBUG: generated sql query for job 1253 and task 10
|
||||||
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102013 lineitem WHERE (l_quantity < 5.0)"
|
||||||
|
DEBUG: generated sql query for job 1253 and task 12
|
||||||
|
DETAIL: query string: "SELECT l_partkey, l_suppkey FROM lineitem_102014 lineitem WHERE (l_quantity < 5.0)"
|
||||||
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
|
DEBUG: assigned task 8 to node localhost:57637
|
||||||
|
DEBUG: assigned task 6 to node localhost:57638
|
||||||
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
|
DEBUG: assigned task 10 to node localhost:57638
|
||||||
DEBUG: generated sql query for job 1254 and task 2
|
DEBUG: generated sql query for job 1254 and task 2
|
||||||
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102016 orders WHERE (o_totalprice <> 4.0)"
|
|
||||||
DEBUG: generated sql query for job 1254 and task 4
|
|
||||||
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102015 orders WHERE (o_totalprice <> 4.0)"
|
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102015 orders WHERE (o_totalprice <> 4.0)"
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
DEBUG: generated sql query for job 1254 and task 4
|
||||||
DEBUG: assigned task 4 to node localhost:57638
|
DETAIL: query string: "SELECT o_orderkey, o_shippriority FROM orders_102016 orders WHERE (o_totalprice <> 4.0)"
|
||||||
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
DEBUG: join prunable for task partitionId 0 and 1
|
DEBUG: join prunable for task partitionId 0 and 1
|
||||||
DEBUG: join prunable for task partitionId 0 and 2
|
DEBUG: join prunable for task partitionId 0 and 2
|
||||||
DEBUG: join prunable for task partitionId 0 and 3
|
DEBUG: join prunable for task partitionId 0 and 3
|
||||||
|
|
|
@ -16,10 +16,10 @@ WHERE
|
||||||
o_custkey = c_custkey;
|
o_custkey = c_custkey;
|
||||||
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
||||||
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [6001,7000]
|
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
||||||
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
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 [1,1000]
|
||||||
|
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
||||||
DEBUG: pruning merge fetch taskId 1
|
DEBUG: pruning merge fetch taskId 1
|
||||||
DETAIL: Creating dependency on merge taskId 5
|
DETAIL: Creating dependency on merge taskId 5
|
||||||
DEBUG: pruning merge fetch taskId 4
|
DEBUG: pruning merge fetch taskId 4
|
||||||
|
@ -40,8 +40,8 @@ FROM
|
||||||
WHERE
|
WHERE
|
||||||
o_custkey = c_custkey AND
|
o_custkey = c_custkey AND
|
||||||
o_orderkey < 0;
|
o_orderkey < 0;
|
||||||
DEBUG: predicate pruning for shardId 102016
|
|
||||||
DEBUG: predicate pruning for shardId 102015
|
DEBUG: predicate pruning for shardId 102015
|
||||||
|
DEBUG: predicate pruning for shardId 102016
|
||||||
count
|
count
|
||||||
-------
|
-------
|
||||||
|
|
||||||
|
@ -56,9 +56,9 @@ FROM
|
||||||
WHERE
|
WHERE
|
||||||
o_custkey = c_custkey AND
|
o_custkey = c_custkey AND
|
||||||
c_custkey < 0;
|
c_custkey < 0;
|
||||||
|
DEBUG: predicate pruning for shardId 102017
|
||||||
DEBUG: predicate pruning for shardId 102034
|
DEBUG: predicate pruning for shardId 102034
|
||||||
DEBUG: predicate pruning for shardId 102033
|
DEBUG: predicate pruning for shardId 102033
|
||||||
DEBUG: predicate pruning for shardId 102017
|
|
||||||
count
|
count
|
||||||
-------
|
-------
|
||||||
|
|
||||||
|
@ -115,12 +115,12 @@ FROM
|
||||||
WHERE
|
WHERE
|
||||||
l_partkey = c_nationkey AND
|
l_partkey = c_nationkey AND
|
||||||
l_orderkey < 0;
|
l_orderkey < 0;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102012
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102012
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
count
|
count
|
||||||
-------
|
-------
|
||||||
|
|
||||||
|
|
|
@ -25,14 +25,14 @@ FROM
|
||||||
WHERE
|
WHERE
|
||||||
o_custkey = c_custkey;
|
o_custkey = c_custkey;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
DEBUG: assigned task 4 to node localhost:57638
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
DEBUG: join prunable for intervals [1,1000] and [1001,2000]
|
||||||
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
DEBUG: join prunable for intervals [1,1000] and [6001,7000]
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [6001,7000]
|
|
||||||
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
DEBUG: join prunable for intervals [1001,2000] and [1,1000]
|
||||||
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
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 [1,1000]
|
||||||
|
DEBUG: join prunable for intervals [6001,7000] and [1001,2000]
|
||||||
DEBUG: pruning merge fetch taskId 1
|
DEBUG: pruning merge fetch taskId 1
|
||||||
DETAIL: Creating dependency on merge taskId 5
|
DETAIL: Creating dependency on merge taskId 5
|
||||||
DEBUG: pruning merge fetch taskId 4
|
DEBUG: pruning merge fetch taskId 4
|
||||||
|
@ -64,40 +64,40 @@ WHERE
|
||||||
o_custkey = c_custkey AND
|
o_custkey = c_custkey AND
|
||||||
o_orderkey = l_orderkey;
|
o_orderkey = l_orderkey;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: assigned task 3 to node localhost:57637
|
|
||||||
DEBUG: assigned task 15 to node localhost:57638
|
|
||||||
DEBUG: assigned task 6 to node localhost:57637
|
|
||||||
DEBUG: assigned task 18 to node localhost:57638
|
|
||||||
DEBUG: assigned task 9 to node localhost:57637
|
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: join prunable for intervals [1,2496] and [13921,14947]
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
DEBUG: join prunable for intervals [1,2496] and [11554,13920]
|
DEBUG: assigned task 6 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,2496] and [8997,11554]
|
DEBUG: assigned task 15 to node localhost:57637
|
||||||
DEBUG: join prunable for intervals [1,2496] and [4965,5986]
|
DEBUG: assigned task 18 to node localhost:57638
|
||||||
DEBUG: join prunable for intervals [1,2496] and [2497,4964]
|
DEBUG: join prunable for intervals [1,2496] and [2497,4964]
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [13921,14947]
|
DEBUG: join prunable for intervals [1,2496] and [4965,5986]
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [11554,13920]
|
DEBUG: join prunable for intervals [1,2496] and [8997,11554]
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,11554]
|
DEBUG: join prunable for intervals [1,2496] and [11554,13920]
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [4965,5986]
|
DEBUG: join prunable for intervals [1,2496] and [13921,14947]
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [1,2496]
|
DEBUG: join prunable for intervals [2497,4964] and [1,2496]
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [13921,14947]
|
DEBUG: join prunable for intervals [2497,4964] and [4965,5986]
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [11554,13920]
|
DEBUG: join prunable for intervals [2497,4964] and [8997,11554]
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,11554]
|
DEBUG: join prunable for intervals [2497,4964] and [11554,13920]
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [2497,4964]
|
DEBUG: join prunable for intervals [2497,4964] and [13921,14947]
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [1,2496]
|
DEBUG: join prunable for intervals [4965,5986] and [1,2496]
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [13921,14947]
|
DEBUG: join prunable for intervals [4965,5986] and [2497,4964]
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [4965,5986]
|
DEBUG: join prunable for intervals [4965,5986] and [8997,11554]
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [2497,4964]
|
DEBUG: join prunable for intervals [4965,5986] and [11554,13920]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [13921,14947]
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,2496]
|
DEBUG: join prunable for intervals [8997,11554] and [1,2496]
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [13921,14947]
|
DEBUG: join prunable for intervals [8997,11554] and [2497,4964]
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [4965,5986]
|
DEBUG: join prunable for intervals [8997,11554] and [4965,5986]
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [2497,4964]
|
DEBUG: join prunable for intervals [8997,11554] and [13921,14947]
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,2496]
|
DEBUG: join prunable for intervals [11554,13920] and [1,2496]
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [11554,13920]
|
DEBUG: join prunable for intervals [11554,13920] and [2497,4964]
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [8997,11554]
|
DEBUG: join prunable for intervals [11554,13920] and [4965,5986]
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [4965,5986]
|
DEBUG: join prunable for intervals [11554,13920] and [13921,14947]
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [2497,4964]
|
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,2496]
|
DEBUG: join prunable for intervals [13921,14947] and [1,2496]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [2497,4964]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [4965,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [8997,11554]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [11554,13920]
|
||||||
DEBUG: pruning merge fetch taskId 1
|
DEBUG: pruning merge fetch taskId 1
|
||||||
DETAIL: Creating dependency on merge taskId 19
|
DETAIL: Creating dependency on merge taskId 19
|
||||||
DEBUG: pruning merge fetch taskId 4
|
DEBUG: pruning merge fetch taskId 4
|
||||||
|
@ -116,10 +116,10 @@ DEBUG: pruning merge fetch taskId 22
|
||||||
DETAIL: Creating dependency on merge taskId 54
|
DETAIL: Creating dependency on merge taskId 54
|
||||||
DEBUG: assigned task 6 to node localhost:57637
|
DEBUG: assigned task 6 to node localhost:57637
|
||||||
DEBUG: assigned task 3 to node localhost:57638
|
DEBUG: assigned task 3 to node localhost:57638
|
||||||
DEBUG: assigned task 24 to node localhost:57637
|
|
||||||
DEBUG: assigned task 9 to node localhost:57638
|
|
||||||
DEBUG: assigned task 12 to node localhost:57637
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
DEBUG: assigned task 18 to node localhost:57638
|
DEBUG: assigned task 9 to node localhost:57638
|
||||||
|
DEBUG: assigned task 18 to node localhost:57637
|
||||||
|
DEBUG: assigned task 24 to node localhost:57638
|
||||||
DEBUG: propagating assignment from merge task 40 to constrained sql task 15
|
DEBUG: propagating assignment from merge task 40 to constrained sql task 15
|
||||||
DEBUG: propagating assignment from merge task 47 to constrained sql task 21
|
DEBUG: propagating assignment from merge task 47 to constrained sql task 21
|
||||||
DEBUG: CommitTransactionCommand
|
DEBUG: CommitTransactionCommand
|
||||||
|
@ -154,15 +154,15 @@ FROM
|
||||||
WHERE
|
WHERE
|
||||||
l_partkey = c_nationkey;
|
l_partkey = c_nationkey;
|
||||||
DEBUG: StartTransactionCommand
|
DEBUG: StartTransactionCommand
|
||||||
DEBUG: assigned task 10 to node localhost:57637
|
|
||||||
DEBUG: assigned task 12 to node localhost:57638
|
|
||||||
DEBUG: assigned task 6 to node localhost:57637
|
|
||||||
DEBUG: assigned task 8 to node localhost:57638
|
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
|
||||||
DEBUG: assigned task 4 to node localhost:57638
|
|
||||||
DEBUG: assigned task 2 to node localhost:57637
|
|
||||||
DEBUG: assigned task 6 to node localhost:57638
|
|
||||||
DEBUG: assigned task 4 to node localhost:57637
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
|
DEBUG: assigned task 8 to node localhost:57637
|
||||||
|
DEBUG: assigned task 6 to node localhost:57638
|
||||||
|
DEBUG: assigned task 12 to node localhost:57637
|
||||||
|
DEBUG: assigned task 10 to node localhost:57638
|
||||||
|
DEBUG: assigned task 4 to node localhost:57637
|
||||||
|
DEBUG: assigned task 2 to node localhost:57638
|
||||||
|
DEBUG: assigned task 6 to node localhost:57637
|
||||||
DEBUG: join prunable for task partitionId 0 and 1
|
DEBUG: join prunable for task partitionId 0 and 1
|
||||||
DEBUG: join prunable for task partitionId 0 and 2
|
DEBUG: join prunable for task partitionId 0 and 2
|
||||||
DEBUG: join prunable for task partitionId 0 and 3
|
DEBUG: join prunable for task partitionId 0 and 3
|
||||||
|
|
|
@ -21,10 +21,10 @@ SELECT shardminvalue, shardmaxvalue from pg_dist_shard WHERE shardid = 102010;
|
||||||
-- Check that partition and join pruning works when min/max values exist
|
-- Check that partition and join pruning works when min/max values exist
|
||||||
-- Adding l_orderkey = 1 to make the query not router executable
|
-- Adding l_orderkey = 1 to make the query not router executable
|
||||||
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030 or l_orderkey = 1;
|
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030 or l_orderkey = 1;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
l_orderkey | l_linenumber | l_shipdate
|
l_orderkey | l_linenumber | l_shipdate
|
||||||
------------+--------------+------------
|
------------+--------------+------------
|
||||||
9030 | 1 | 09-02-1998
|
9030 | 1 | 09-02-1998
|
||||||
|
@ -43,12 +43,12 @@ DEBUG: predicate pruning for shardId 102010
|
||||||
|
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey;
|
WHERE l_orderkey = o_orderkey;
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
36086 | 3.0076679446574429
|
36086 | 3.0076679446574429
|
||||||
|
@ -58,10 +58,10 @@ DEBUG: join prunable for intervals [1,2496] and [8997,14946]
|
||||||
-- partition or join pruning for the shard with null min value.
|
-- partition or join pruning for the shard with null min value.
|
||||||
UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 102009;
|
UPDATE pg_dist_shard SET shardminvalue = NULL WHERE shardid = 102009;
|
||||||
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030;
|
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
l_orderkey | l_linenumber | l_shipdate
|
l_orderkey | l_linenumber | l_shipdate
|
||||||
------------+--------------+------------
|
------------+--------------+------------
|
||||||
9030 | 1 | 09-02-1998
|
9030 | 1 | 09-02-1998
|
||||||
|
@ -74,11 +74,11 @@ DEBUG: predicate pruning for shardId 102010
|
||||||
|
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey;
|
WHERE l_orderkey = o_orderkey;
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
36086 | 3.0076679446574429
|
36086 | 3.0076679446574429
|
||||||
|
@ -88,9 +88,9 @@ DEBUG: join prunable for intervals [2497,4964] and [8997,14946]
|
||||||
-- don't apply partition or join pruning for this other shard either.
|
-- don't apply partition or join pruning for this other shard either.
|
||||||
UPDATE pg_dist_shard SET shardmaxvalue = NULL WHERE shardid = 102010;
|
UPDATE pg_dist_shard SET shardmaxvalue = NULL WHERE shardid = 102010;
|
||||||
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030;
|
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
l_orderkey | l_linenumber | l_shipdate
|
l_orderkey | l_linenumber | l_shipdate
|
||||||
------------+--------------+------------
|
------------+--------------+------------
|
||||||
9030 | 1 | 09-02-1998
|
9030 | 1 | 09-02-1998
|
||||||
|
@ -103,10 +103,10 @@ DEBUG: predicate pruning for shardId 102011
|
||||||
|
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey;
|
WHERE l_orderkey = o_orderkey;
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
36086 | 3.0076679446574429
|
36086 | 3.0076679446574429
|
||||||
|
@ -116,10 +116,10 @@ DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
-- should apply partition and join pruning for this shard now.
|
-- should apply partition and join pruning for this shard now.
|
||||||
UPDATE pg_dist_shard SET shardminvalue = '0' WHERE shardid = 102009;
|
UPDATE pg_dist_shard SET shardminvalue = '0' WHERE shardid = 102009;
|
||||||
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030;
|
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
l_orderkey | l_linenumber | l_shipdate
|
l_orderkey | l_linenumber | l_shipdate
|
||||||
------------+--------------+------------
|
------------+--------------+------------
|
||||||
9030 | 1 | 09-02-1998
|
9030 | 1 | 09-02-1998
|
||||||
|
@ -132,11 +132,11 @@ DEBUG: predicate pruning for shardId 102009
|
||||||
|
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem, orders
|
||||||
WHERE l_orderkey = o_orderkey;
|
WHERE l_orderkey = o_orderkey;
|
||||||
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
|
||||||
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
|
||||||
DEBUG: join prunable for intervals [0,2496] and [8997,14946]
|
DEBUG: join prunable for intervals [0,2496] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [4965,5986] and [8997,14946]
|
||||||
|
DEBUG: join prunable for intervals [8997,11554] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [11554,13920] and [1,5986]
|
||||||
|
DEBUG: join prunable for intervals [13921,14947] and [1,5986]
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
36086 | 3.0076679446574429
|
36086 | 3.0076679446574429
|
||||||
|
|
|
@ -6,10 +6,10 @@
|
||||||
SET client_min_messages TO DEBUG2;
|
SET client_min_messages TO DEBUG2;
|
||||||
-- Adding additional l_orderkey = 1 to make this query not router executable
|
-- Adding additional l_orderkey = 1 to make this query not router executable
|
||||||
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030 or l_orderkey = 1;
|
SELECT l_orderkey, l_linenumber, l_shipdate FROM lineitem WHERE l_orderkey = 9030 or l_orderkey = 1;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
l_orderkey | l_linenumber | l_shipdate
|
l_orderkey | l_linenumber | l_shipdate
|
||||||
------------+--------------+------------
|
------------+--------------+------------
|
||||||
9030 | 1 | 09-02-1998
|
9030 | 1 | 09-02-1998
|
||||||
|
@ -32,9 +32,9 @@ DEBUG: predicate pruning for shardId 102010
|
||||||
-- trigger the the creation of toasted tables and indexes. This in turn prints
|
-- trigger the the creation of toasted tables and indexes. This in turn prints
|
||||||
-- non-deterministic debug messages. To avoid this chain, we use l_linenumber.
|
-- non-deterministic debug messages. To avoid this chain, we use l_linenumber.
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem WHERE l_orderkey > 9030;
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
sum | avg
|
sum | avg
|
||||||
-------+--------------------
|
-------+--------------------
|
||||||
17999 | 3.0189533713518953
|
17999 | 3.0189533713518953
|
||||||
|
@ -50,12 +50,12 @@ DEBUG: predicate pruning for shardId 102011
|
||||||
|
|
||||||
-- The following query should prune out all shards and return empty results
|
-- The following query should prune out all shards and return empty results
|
||||||
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem WHERE l_orderkey > 20000;
|
SELECT sum(l_linenumber), avg(l_linenumber) FROM lineitem WHERE l_orderkey > 20000;
|
||||||
DEBUG: predicate pruning for shardId 102014
|
|
||||||
DEBUG: predicate pruning for shardId 102013
|
|
||||||
DEBUG: predicate pruning for shardId 102012
|
|
||||||
DEBUG: predicate pruning for shardId 102011
|
|
||||||
DEBUG: predicate pruning for shardId 102010
|
|
||||||
DEBUG: predicate pruning for shardId 102009
|
DEBUG: predicate pruning for shardId 102009
|
||||||
|
DEBUG: predicate pruning for shardId 102010
|
||||||
|
DEBUG: predicate pruning for shardId 102011
|
||||||
|
DEBUG: predicate pruning for shardId 102012
|
||||||
|
DEBUG: predicate pruning for shardId 102013
|
||||||
|
DEBUG: predicate pruning for shardId 102014
|
||||||
sum | avg
|
sum | avg
|
||||||
-----+-----
|
-----+-----
|
||||||
|
|
|
|
||||||
|
|
|
@ -21,6 +21,10 @@ CREATE FUNCTION debug_equality_expression(regclass)
|
||||||
RETURNS cstring
|
RETURNS cstring
|
||||||
AS 'citus'
|
AS 'citus'
|
||||||
LANGUAGE C STRICT;
|
LANGUAGE C STRICT;
|
||||||
|
CREATE FUNCTION print_sorted_shard_intervals(regclass)
|
||||||
|
RETURNS text[]
|
||||||
|
AS 'citus'
|
||||||
|
LANGUAGE C STRICT;
|
||||||
-- ===================================================================
|
-- ===================================================================
|
||||||
-- test shard pruning functionality
|
-- test shard pruning functionality
|
||||||
-- ===================================================================
|
-- ===================================================================
|
||||||
|
@ -40,7 +44,7 @@ VALUES
|
||||||
SELECT prune_using_no_values('pruning');
|
SELECT prune_using_no_values('pruning');
|
||||||
prune_using_no_values
|
prune_using_no_values
|
||||||
-----------------------
|
-----------------------
|
||||||
{13,12,11,10}
|
{10,11,12,13}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- with a single value, expect a single shard
|
-- with a single value, expect a single shard
|
||||||
|
@ -61,7 +65,7 @@ SELECT prune_using_single_value('pruning', NULL);
|
||||||
SELECT prune_using_either_value('pruning', 'tomato', 'petunia');
|
SELECT prune_using_either_value('pruning', 'tomato', 'petunia');
|
||||||
prune_using_either_value
|
prune_using_either_value
|
||||||
--------------------------
|
--------------------------
|
||||||
{12,11}
|
{11,12}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
-- an AND clause with incompatible values returns no shards
|
-- an AND clause with incompatible values returns no shards
|
||||||
|
@ -85,3 +89,88 @@ SELECT debug_equality_expression('pruning');
|
||||||
{OPEXPR :opno 98 :opfuncid 67 :opresulttype 16 :opretset false :opcollid 0 :inputcollid 100 :args ({VAR :varno 1 :varattno 1 :vartype 25 :vartypmod -1 :varcollid 100 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} {CONST :consttype 25 :consttypmod -1 :constcollid 100 :constlen -1 :constbyval false :constisnull true :location -1 :constvalue <>}) :location -1}
|
{OPEXPR :opno 98 :opfuncid 67 :opresulttype 16 :opretset false :opcollid 0 :inputcollid 100 :args ({VAR :varno 1 :varattno 1 :vartype 25 :vartypmod -1 :varcollid 100 :varlevelsup 0 :varnoold 1 :varoattno 1 :location -1} {CONST :consttype 25 :consttypmod -1 :constcollid 100 :constlen -1 :constbyval false :constisnull true :location -1 :constvalue <>}) :location -1}
|
||||||
(1 row)
|
(1 row)
|
||||||
|
|
||||||
|
-- print the initial ordering of shard intervals
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,11,12,13}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- update only min value for one shard
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 11;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,12,13,11}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 12;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,13,11,12}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 10;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{13,10,11,12}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- all shard placements are uninitialized
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 13;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,11,12,13}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now update the metadata so that the table is a range distributed table
|
||||||
|
UPDATE pg_dist_partition SET partmethod = 'r' WHERE logicalrelid = 'pruning'::regclass;
|
||||||
|
-- now the comparison is done via the partition column type, which is text
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'a', shardmaxvalue = 'b' WHERE shardid = 10;
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'c', shardmaxvalue = 'd' WHERE shardid = 11;
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'e', shardmaxvalue = 'f' WHERE shardid = 12;
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'g', shardmaxvalue = 'h' WHERE shardid = 13;
|
||||||
|
-- print the ordering of shard intervals with range partitioning as well
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,11,12,13}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- update only min value for one shard
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 11;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,12,13,11}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 12;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,13,11,12}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 10;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{13,10,11,12}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
-- all shard placements are uninitialized
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 13;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
print_sorted_shard_intervals
|
||||||
|
------------------------------
|
||||||
|
{10,11,12,13}
|
||||||
|
(1 row)
|
||||||
|
|
||||||
|
|
|
@ -105,8 +105,8 @@ DEBUG: Creating router plan
|
||||||
DEBUG: Plan is router executable
|
DEBUG: Plan is router executable
|
||||||
-- first, test zero-shard SELECT, which should return an empty row
|
-- first, test zero-shard SELECT, which should return an empty row
|
||||||
SELECT COUNT(*) FROM articles_hash WHERE author_id = 1 AND author_id = 2;
|
SELECT COUNT(*) FROM articles_hash WHERE author_id = 1 AND author_id = 2;
|
||||||
DEBUG: predicate pruning for shardId 103301
|
|
||||||
DEBUG: predicate pruning for shardId 103300
|
DEBUG: predicate pruning for shardId 103300
|
||||||
|
DEBUG: predicate pruning for shardId 103301
|
||||||
count
|
count
|
||||||
-------
|
-------
|
||||||
|
|
||||||
|
@ -1120,7 +1120,6 @@ DEBUG: predicate pruning for shardId 103301
|
||||||
-- insert query is router plannable even under task-tracker
|
-- insert query is router plannable even under task-tracker
|
||||||
INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814);
|
INSERT INTO articles_hash VALUES (51, 1, 'amateus', 1814);
|
||||||
DEBUG: Creating router plan
|
DEBUG: Creating router plan
|
||||||
DEBUG: predicate pruning for shardId 103301
|
|
||||||
DEBUG: Plan is router executable
|
DEBUG: Plan is router executable
|
||||||
-- verify insert is successfull (not router plannable and executable)
|
-- verify insert is successfull (not router plannable and executable)
|
||||||
SELECT id
|
SELECT id
|
||||||
|
|
|
@ -260,8 +260,8 @@ ALTER TABLE IF EXISTS non_existent_table ADD COLUMN new_column INTEGER;
|
||||||
NOTICE: relation "non_existent_table" does not exist, skipping
|
NOTICE: relation "non_existent_table" does not exist, skipping
|
||||||
ALTER TABLE IF EXISTS lineitem_alter ALTER COLUMN int_column2 SET DATA TYPE INTEGER;
|
ALTER TABLE IF EXISTS lineitem_alter ALTER COLUMN int_column2 SET DATA TYPE INTEGER;
|
||||||
ALTER TABLE lineitem_alter DROP COLUMN non_existent_column;
|
ALTER TABLE lineitem_alter DROP COLUMN non_existent_column;
|
||||||
WARNING: could not receive query results from localhost:57637
|
WARNING: could not receive query results from localhost:57638
|
||||||
DETAIL: Client error: column "non_existent_column" of relation "lineitem_alter_103009" does not exist
|
DETAIL: Client error: column "non_existent_column" of relation "lineitem_alter_103000" does not exist
|
||||||
ERROR: could not execute DDL command on worker node shards
|
ERROR: could not execute DDL command on worker node shards
|
||||||
ALTER TABLE lineitem_alter DROP COLUMN IF EXISTS non_existent_column;
|
ALTER TABLE lineitem_alter DROP COLUMN IF EXISTS non_existent_column;
|
||||||
NOTICE: column "non_existent_column" of relation "lineitem_alter" does not exist, skipping
|
NOTICE: column "non_existent_column" of relation "lineitem_alter" does not exist, skipping
|
||||||
|
@ -360,15 +360,15 @@ DETAIL: Only ADD|DROP COLUMN, SET|DROP NOT NULL, SET|DROP DEFAULT and TYPE subc
|
||||||
-- Verify that we error out in case of postgres errors on supported statement
|
-- Verify that we error out in case of postgres errors on supported statement
|
||||||
-- types
|
-- types
|
||||||
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type;
|
ALTER TABLE lineitem_alter ADD COLUMN new_column non_existent_type;
|
||||||
WARNING: could not receive query results from localhost:57637
|
WARNING: could not receive query results from localhost:57638
|
||||||
DETAIL: Client error: type "non_existent_type" does not exist
|
DETAIL: Client error: type "non_existent_type" does not exist
|
||||||
ERROR: could not execute DDL command on worker node shards
|
ERROR: could not execute DDL command on worker node shards
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN null_column SET NOT NULL;
|
ALTER TABLE lineitem_alter ALTER COLUMN null_column SET NOT NULL;
|
||||||
WARNING: could not receive query results from localhost:57637
|
WARNING: could not receive query results from localhost:57638
|
||||||
DETAIL: Client error: column "null_column" contains null values
|
DETAIL: Client error: column "null_column" contains null values
|
||||||
ERROR: could not execute DDL command on worker node shards
|
ERROR: could not execute DDL command on worker node shards
|
||||||
ALTER TABLE lineitem_alter ALTER COLUMN l_partkey SET DEFAULT 'a';
|
ALTER TABLE lineitem_alter ALTER COLUMN l_partkey SET DEFAULT 'a';
|
||||||
WARNING: could not receive query results from localhost:57637
|
WARNING: could not receive query results from localhost:57638
|
||||||
DETAIL: Client error: invalid input syntax for integer: "a"
|
DETAIL: Client error: invalid input syntax for integer: "a"
|
||||||
ERROR: could not execute DDL command on worker node shards
|
ERROR: could not execute DDL command on worker node shards
|
||||||
-- Verify that we error out on statements involving RENAME
|
-- Verify that we error out on statements involving RENAME
|
||||||
|
|
|
@ -349,9 +349,9 @@ SET client_min_messages TO DEBUG2;
|
||||||
SELECT * FROM
|
SELECT * FROM
|
||||||
(SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE a = 'onder' GROUP BY a)
|
(SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE a = 'onder' GROUP BY a)
|
||||||
AS foo;
|
AS foo;
|
||||||
DEBUG: predicate pruning for shardId 102029
|
|
||||||
DEBUG: predicate pruning for shardId 102027
|
|
||||||
DEBUG: predicate pruning for shardId 102026
|
DEBUG: predicate pruning for shardId 102026
|
||||||
|
DEBUG: predicate pruning for shardId 102027
|
||||||
|
DEBUG: predicate pruning for shardId 102029
|
||||||
count
|
count
|
||||||
-------
|
-------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
@ -359,9 +359,9 @@ DEBUG: predicate pruning for shardId 102026
|
||||||
SELECT * FROM
|
SELECT * FROM
|
||||||
(SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a)
|
(SELECT count(*) FROM subquery_pruning_varchar_test_table WHERE 'eren' = a GROUP BY a)
|
||||||
AS foo;
|
AS foo;
|
||||||
DEBUG: predicate pruning for shardId 102029
|
|
||||||
DEBUG: predicate pruning for shardId 102028
|
|
||||||
DEBUG: predicate pruning for shardId 102026
|
DEBUG: predicate pruning for shardId 102026
|
||||||
|
DEBUG: predicate pruning for shardId 102028
|
||||||
|
DEBUG: predicate pruning for shardId 102029
|
||||||
count
|
count
|
||||||
-------
|
-------
|
||||||
(0 rows)
|
(0 rows)
|
||||||
|
|
|
@ -27,6 +27,11 @@ CREATE FUNCTION debug_equality_expression(regclass)
|
||||||
AS 'citus'
|
AS 'citus'
|
||||||
LANGUAGE C STRICT;
|
LANGUAGE C STRICT;
|
||||||
|
|
||||||
|
CREATE FUNCTION print_sorted_shard_intervals(regclass)
|
||||||
|
RETURNS text[]
|
||||||
|
AS 'citus'
|
||||||
|
LANGUAGE C STRICT;
|
||||||
|
|
||||||
-- ===================================================================
|
-- ===================================================================
|
||||||
-- test shard pruning functionality
|
-- test shard pruning functionality
|
||||||
-- ===================================================================
|
-- ===================================================================
|
||||||
|
@ -66,3 +71,50 @@ SELECT prune_using_both_values('pruning', 'tomato', 'rose');
|
||||||
|
|
||||||
-- unit test of the equality expression generation code
|
-- unit test of the equality expression generation code
|
||||||
SELECT debug_equality_expression('pruning');
|
SELECT debug_equality_expression('pruning');
|
||||||
|
|
||||||
|
-- print the initial ordering of shard intervals
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- update only min value for one shard
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 11;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 12;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 10;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- all shard placements are uninitialized
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 13;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- now update the metadata so that the table is a range distributed table
|
||||||
|
UPDATE pg_dist_partition SET partmethod = 'r' WHERE logicalrelid = 'pruning'::regclass;
|
||||||
|
|
||||||
|
-- now the comparison is done via the partition column type, which is text
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'a', shardmaxvalue = 'b' WHERE shardid = 10;
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'c', shardmaxvalue = 'd' WHERE shardid = 11;
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'e', shardmaxvalue = 'f' WHERE shardid = 12;
|
||||||
|
UPDATE pg_dist_shard SET shardminvalue = 'g', shardmaxvalue = 'h' WHERE shardid = 13;
|
||||||
|
|
||||||
|
-- print the ordering of shard intervals with range partitioning as well
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- update only min value for one shard
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 11;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 12;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- now lets have one more shard without min/max values
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 10;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
||||||
|
-- all shard placements are uninitialized
|
||||||
|
UPDATE pg_dist_shard set shardminvalue = NULL, shardmaxvalue = NULL WHERE shardid = 13;
|
||||||
|
SELECT print_sorted_shard_intervals('pruning');
|
||||||
|
|
Loading…
Reference in New Issue