mirror of https://github.com/citusdata/citus.git
Rename DistTableCacheEntry to CitusTableCacheEntry
parent
b514ab0f55
commit
a7cca1bcde
|
@ -97,7 +97,7 @@ CallFuncExprRemotely(CallStmt *callStmt, DistObjectCacheEntry *procedure,
|
|||
return false;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *distTable = CitusTableCacheEntry(colocatedRelationId);
|
||||
CitusTableCacheEntry *distTable = LookupCitusTableCacheEntry(colocatedRelationId);
|
||||
Var *partitionColumn = distTable->partitionColumn;
|
||||
if (partitionColumn == NULL)
|
||||
{
|
||||
|
|
|
@ -793,7 +793,7 @@ static void
|
|||
EnsureTableCanBeColocatedWith(Oid relationId, char replicationModel,
|
||||
Oid distributionColumnType, Oid sourceRelationId)
|
||||
{
|
||||
DistTableCacheEntry *sourceTableEntry = CitusTableCacheEntry(sourceRelationId);
|
||||
CitusTableCacheEntry *sourceTableEntry = LookupCitusTableCacheEntry(sourceRelationId);
|
||||
char sourceDistributionMethod = sourceTableEntry->partitionMethod;
|
||||
char sourceReplicationModel = sourceTableEntry->replicationModel;
|
||||
Var *sourceDistributionColumn = ForceDistPartitionKey(sourceRelationId);
|
||||
|
|
|
@ -410,7 +410,7 @@ static void
|
|||
EnsureFunctionCanBeColocatedWithTable(Oid functionOid, Oid distributionColumnType,
|
||||
Oid sourceRelationId)
|
||||
{
|
||||
DistTableCacheEntry *sourceTableEntry = CitusTableCacheEntry(sourceRelationId);
|
||||
CitusTableCacheEntry *sourceTableEntry = LookupCitusTableCacheEntry(sourceRelationId);
|
||||
char sourceDistributionMethod = sourceTableEntry->partitionMethod;
|
||||
char sourceReplicationModel = sourceTableEntry->replicationModel;
|
||||
|
||||
|
|
|
@ -2018,7 +2018,7 @@ CitusCopyDestReceiverStartup(DestReceiver *dest, int operation,
|
|||
|
||||
/* look up table properties */
|
||||
Relation distributedRelation = heap_open(tableId, RowExclusiveLock);
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(tableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(tableId);
|
||||
partitionMethod = cacheEntry->partitionMethod;
|
||||
|
||||
copyDest->distributedRelation = distributedRelation;
|
||||
|
|
|
@ -189,7 +189,7 @@ LockTruncatedRelationMetadataInWorkers(TruncateStmt *truncateStatement)
|
|||
|
||||
distributedRelationList = lappend_oid(distributedRelationList, relationId);
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
Assert(cacheEntry != NULL);
|
||||
|
||||
List *referencingTableList = cacheEntry->referencingRelationsViaForeignKey;
|
||||
|
|
|
@ -57,10 +57,10 @@ typedef struct NodeToNodeFragmentsTransfer
|
|||
/* forward declarations of local functions */
|
||||
static void WrapTasksForPartitioning(const char *resultIdPrefix, List *selectTaskList,
|
||||
int partitionColumnIndex,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
bool binaryFormat);
|
||||
static List * ExecutePartitionTaskList(List *partitionTaskList,
|
||||
DistTableCacheEntry *targetRelation);
|
||||
CitusTableCacheEntry *targetRelation);
|
||||
static ArrayType * CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int
|
||||
datumCount, Oid typeId);
|
||||
static void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shardCount,
|
||||
|
@ -68,14 +68,14 @@ static void ShardMinMaxValueArrays(ShardInterval **shardIntervalArray, int shard
|
|||
ArrayType **maxValueArray);
|
||||
static char * SourceShardPrefix(const char *resultPrefix, uint64 shardId);
|
||||
static DistributedResultFragment * TupleToDistributedResultFragment(
|
||||
TupleTableSlot *tupleSlot, DistTableCacheEntry *targetRelation);
|
||||
TupleTableSlot *tupleSlot, CitusTableCacheEntry *targetRelation);
|
||||
static Tuplestorestate * ExecuteSelectTasksIntoTupleStore(List *taskList,
|
||||
TupleDesc resultDescriptor,
|
||||
bool errorOnAnyFailure);
|
||||
static List ** ColocateFragmentsWithRelation(List *fragmentList,
|
||||
DistTableCacheEntry *targetRelation);
|
||||
CitusTableCacheEntry *targetRelation);
|
||||
static List * ColocationTransfers(List *fragmentList,
|
||||
DistTableCacheEntry *targetRelation);
|
||||
CitusTableCacheEntry *targetRelation);
|
||||
static List * FragmentTransferTaskList(List *fragmentListTransfers);
|
||||
static char * QueryStringForFragmentsTransfer(
|
||||
NodeToNodeFragmentsTransfer *fragmentsTransfer);
|
||||
|
@ -99,7 +99,7 @@ static void ExecuteFetchTaskList(List *fetchTaskList);
|
|||
List **
|
||||
RedistributeTaskListResults(const char *resultIdPrefix, List *selectTaskList,
|
||||
int partitionColumnIndex,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
bool binaryFormat)
|
||||
{
|
||||
/*
|
||||
|
@ -134,7 +134,7 @@ RedistributeTaskListResults(const char *resultIdPrefix, List *selectTaskList,
|
|||
List *
|
||||
PartitionTasklistResults(const char *resultIdPrefix, List *selectTaskList,
|
||||
int partitionColumnIndex,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
bool binaryFormat)
|
||||
{
|
||||
if (targetRelation->partitionMethod != DISTRIBUTE_BY_HASH &&
|
||||
|
@ -168,7 +168,7 @@ PartitionTasklistResults(const char *resultIdPrefix, List *selectTaskList,
|
|||
static void
|
||||
WrapTasksForPartitioning(const char *resultIdPrefix, List *selectTaskList,
|
||||
int partitionColumnIndex,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
bool binaryFormat)
|
||||
{
|
||||
ShardInterval **shardIntervalArray = targetRelation->sortedShardIntervalArray;
|
||||
|
@ -318,7 +318,7 @@ CreateArrayFromDatums(Datum *datumArray, bool *nullsArray, int datumCount, Oid t
|
|||
* and returns its results as a list of DistributedResultFragment.
|
||||
*/
|
||||
static List *
|
||||
ExecutePartitionTaskList(List *taskList, DistTableCacheEntry *targetRelation)
|
||||
ExecutePartitionTaskList(List *taskList, CitusTableCacheEntry *targetRelation)
|
||||
{
|
||||
TupleDesc resultDescriptor = NULL;
|
||||
Tuplestorestate *resultStore = NULL;
|
||||
|
@ -366,7 +366,7 @@ ExecutePartitionTaskList(List *taskList, DistTableCacheEntry *targetRelation)
|
|||
*/
|
||||
static DistributedResultFragment *
|
||||
TupleToDistributedResultFragment(TupleTableSlot *tupleSlot,
|
||||
DistTableCacheEntry *targetRelation)
|
||||
CitusTableCacheEntry *targetRelation)
|
||||
{
|
||||
bool isNull = false;
|
||||
uint32 sourceNodeId = DatumGetUInt32(slot_getattr(tupleSlot, 1, &isNull));
|
||||
|
@ -429,7 +429,7 @@ ExecuteSelectTasksIntoTupleStore(List *taskList, TupleDesc resultDescriptor,
|
|||
* done.
|
||||
*/
|
||||
static List **
|
||||
ColocateFragmentsWithRelation(List *fragmentList, DistTableCacheEntry *targetRelation)
|
||||
ColocateFragmentsWithRelation(List *fragmentList, CitusTableCacheEntry *targetRelation)
|
||||
{
|
||||
List *fragmentListTransfers = ColocationTransfers(fragmentList, targetRelation);
|
||||
List *fragmentTransferTaskList = FragmentTransferTaskList(fragmentListTransfers);
|
||||
|
@ -458,7 +458,7 @@ ColocateFragmentsWithRelation(List *fragmentList, DistTableCacheEntry *targetRel
|
|||
* target relations. This prunes away transfers with same source and target
|
||||
*/
|
||||
static List *
|
||||
ColocationTransfers(List *fragmentList, DistTableCacheEntry *targetRelation)
|
||||
ColocationTransfers(List *fragmentList, CitusTableCacheEntry *targetRelation)
|
||||
{
|
||||
HASHCTL transferHashInfo;
|
||||
MemSet(&transferHashInfo, 0, sizeof(HASHCTL));
|
||||
|
|
|
@ -73,7 +73,7 @@ static int PartitionColumnIndexFromColumnList(Oid relationId, List *columnNameLi
|
|||
static List * AddInsertSelectCasts(List *insertTargetList, List *selectTargetList,
|
||||
Oid targetRelationId);
|
||||
static List * RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
List **redistributedResults,
|
||||
bool useBinaryFormat);
|
||||
static int PartitionColumnIndex(List *insertTargetList, Var *partitionColumn);
|
||||
|
@ -219,8 +219,8 @@ CoordinatorInsertSelectExecScanInternal(CustomScanState *node)
|
|||
distSelectJob->jobId);
|
||||
char *distResultPrefix = distResultPrefixString->data;
|
||||
|
||||
DistTableCacheEntry *targetRelation =
|
||||
CitusTableCacheEntry(targetRelationId);
|
||||
CitusTableCacheEntry *targetRelation =
|
||||
LookupCitusTableCacheEntry(targetRelationId);
|
||||
|
||||
int partitionColumnIndex =
|
||||
PartitionColumnIndex(insertTargetList, targetRelation->partitionColumn);
|
||||
|
@ -484,7 +484,7 @@ TwoPhaseInsertSelectTaskList(Oid targetRelationId, Query *insertSelectQuery,
|
|||
RangeTblEntry *insertRte = ExtractResultRelationRTE(insertResultQuery);
|
||||
RangeTblEntry *selectRte = ExtractSelectRangeTableEntry(insertResultQuery);
|
||||
|
||||
DistTableCacheEntry *targetCacheEntry = CitusTableCacheEntry(targetRelationId);
|
||||
CitusTableCacheEntry *targetCacheEntry = LookupCitusTableCacheEntry(targetRelationId);
|
||||
int shardCount = targetCacheEntry->shardIntervalArrayLength;
|
||||
uint32 taskIdIndex = 1;
|
||||
uint64 jobId = INVALID_JOB_ID;
|
||||
|
@ -895,7 +895,7 @@ CastExpr(Expr *expr, Oid sourceType, Oid targetType, Oid targetCollation,
|
|||
bool
|
||||
IsSupportedRedistributionTarget(Oid targetRelationId)
|
||||
{
|
||||
DistTableCacheEntry *tableEntry = CitusTableCacheEntry(targetRelationId);
|
||||
CitusTableCacheEntry *tableEntry = LookupCitusTableCacheEntry(targetRelationId);
|
||||
|
||||
/* only range and hash-distributed tables are currently supported */
|
||||
if (tableEntry->partitionMethod != DISTRIBUTE_BY_HASH &&
|
||||
|
@ -917,7 +917,7 @@ IsSupportedRedistributionTarget(Oid targetRelationId)
|
|||
*/
|
||||
static List *
|
||||
RedistributedInsertSelectTaskList(Query *insertSelectQuery,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
List **redistributedResults,
|
||||
bool useBinaryFormat)
|
||||
{
|
||||
|
|
|
@ -50,7 +50,7 @@ typedef struct PartitionedResultDestReceiver
|
|||
bool binaryCopy;
|
||||
|
||||
/* used for deciding which partition a shard belongs to. */
|
||||
DistTableCacheEntry *shardSearchInfo;
|
||||
CitusTableCacheEntry *shardSearchInfo;
|
||||
|
||||
MemoryContext perTupleContext;
|
||||
|
||||
|
@ -71,10 +71,10 @@ typedef struct PartitionedResultDestReceiver
|
|||
} PartitionedResultDestReceiver;
|
||||
|
||||
static Portal StartPortalForQueryExecution(const char *queryString);
|
||||
static DistTableCacheEntry * QueryTupleShardSearchInfo(ArrayType *minValuesArray,
|
||||
ArrayType *maxValuesArray,
|
||||
char partitionMethod,
|
||||
Var *partitionColumn);
|
||||
static CitusTableCacheEntry * QueryTupleShardSearchInfo(ArrayType *minValuesArray,
|
||||
ArrayType *maxValuesArray,
|
||||
char partitionMethod,
|
||||
Var *partitionColumn);
|
||||
static PartitionedResultDestReceiver * CreatePartitionedResultDestReceiver(char *resultId,
|
||||
int
|
||||
partitionColumnIndex,
|
||||
|
@ -83,7 +83,7 @@ static PartitionedResultDestReceiver * CreatePartitionedResultDestReceiver(char
|
|||
TupleDesc
|
||||
tupleDescriptor,
|
||||
bool binaryCopy,
|
||||
DistTableCacheEntry
|
||||
CitusTableCacheEntry
|
||||
*
|
||||
shardSearchInfo,
|
||||
MemoryContext
|
||||
|
@ -193,8 +193,8 @@ worker_partition_query_result(PG_FUNCTION_ARGS)
|
|||
partitionColumnAttr->atttypmod,
|
||||
partitionColumnAttr->attcollation, 0);
|
||||
|
||||
/* construct an artificial DistTableCacheEntry for shard pruning */
|
||||
DistTableCacheEntry *shardSearchInfo =
|
||||
/* construct an artificial CitusTableCacheEntry for shard pruning */
|
||||
CitusTableCacheEntry *shardSearchInfo =
|
||||
QueryTupleShardSearchInfo(minValuesArray, maxValuesArray,
|
||||
partitionMethod, partitionColumn);
|
||||
|
||||
|
@ -274,11 +274,11 @@ StartPortalForQueryExecution(const char *queryString)
|
|||
|
||||
|
||||
/*
|
||||
* QueryTupleShardSearchInfo returns a DistTableCacheEntry which has enough
|
||||
* QueryTupleShardSearchInfo returns a CitusTableCacheEntry which has enough
|
||||
* information so that FindShardInterval() can find the shard corresponding
|
||||
* to a tuple.
|
||||
*/
|
||||
static DistTableCacheEntry *
|
||||
static CitusTableCacheEntry *
|
||||
QueryTupleShardSearchInfo(ArrayType *minValuesArray, ArrayType *maxValuesArray,
|
||||
char partitionMethod, Var *partitionColumn)
|
||||
{
|
||||
|
@ -335,7 +335,7 @@ QueryTupleShardSearchInfo(ArrayType *minValuesArray, ArrayType *maxValuesArray,
|
|||
shardIntervalArray[partitionIndex]->shardIndex = partitionIndex;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *result = palloc0(sizeof(DistTableCacheEntry));
|
||||
CitusTableCacheEntry *result = palloc0(sizeof(CitusTableCacheEntry));
|
||||
result->partitionMethod = partitionMethod;
|
||||
result->partitionColumn = partitionColumn;
|
||||
result->shardIntervalCompareFunction = shardIntervalCompare;
|
||||
|
@ -364,7 +364,8 @@ QueryTupleShardSearchInfo(ArrayType *minValuesArray, ArrayType *maxValuesArray,
|
|||
static PartitionedResultDestReceiver *
|
||||
CreatePartitionedResultDestReceiver(char *resultIdPrefix, int partitionColumnIndex,
|
||||
int partitionCount, TupleDesc tupleDescriptor,
|
||||
bool binaryCopy, DistTableCacheEntry *shardSearchInfo,
|
||||
bool binaryCopy,
|
||||
CitusTableCacheEntry *shardSearchInfo,
|
||||
MemoryContext perTupleContext)
|
||||
{
|
||||
PartitionedResultDestReceiver *resultDest =
|
||||
|
|
|
@ -106,7 +106,7 @@ void
|
|||
CreateShardsWithRoundRobinPolicy(Oid distributedTableId, int32 shardCount,
|
||||
int32 replicationFactor, bool useExclusiveConnections)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
bool colocatedShard = false;
|
||||
List *insertedShardPlacements = NIL;
|
||||
|
||||
|
|
|
@ -246,7 +246,7 @@ DistributedTableSizeOnWorker(WorkerNode *workerNode, Oid relationId, char *sizeQ
|
|||
List *
|
||||
GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId)
|
||||
{
|
||||
DistTableCacheEntry *distTableCacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *distTableCacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *resultList = NIL;
|
||||
|
||||
int shardIntervalArrayLength = distTableCacheEntry->shardIntervalArrayLength;
|
||||
|
@ -284,7 +284,7 @@ GroupShardPlacementsForTableOnGroup(Oid relationId, int32 groupId)
|
|||
static List *
|
||||
ShardIntervalsOnWorkerGroup(WorkerNode *workerNode, Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *distTableCacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *distTableCacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *shardIntervalList = NIL;
|
||||
int shardIntervalArrayLength = distTableCacheEntry->shardIntervalArrayLength;
|
||||
|
||||
|
@ -486,7 +486,7 @@ TableShardReplicationFactor(Oid relationId)
|
|||
List *
|
||||
LoadShardIntervalList(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *shardList = NIL;
|
||||
|
||||
for (int i = 0; i < cacheEntry->shardIntervalArrayLength; i++)
|
||||
|
@ -511,7 +511,7 @@ LoadShardIntervalList(Oid relationId)
|
|||
int
|
||||
ShardIntervalCount(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
int shardIntervalCount = 0;
|
||||
|
||||
if (cacheEntry->isCitusTable)
|
||||
|
@ -533,7 +533,7 @@ ShardIntervalCount(Oid relationId)
|
|||
List *
|
||||
LoadShardList(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *shardList = NIL;
|
||||
|
||||
for (int i = 0; i < cacheEntry->shardIntervalArrayLength; i++)
|
||||
|
|
|
@ -103,7 +103,7 @@ master_get_table_metadata(PG_FUNCTION_ARGS)
|
|||
CheckCitusVersion(ERROR);
|
||||
|
||||
/* find partition tuple for partitioned relation */
|
||||
DistTableCacheEntry *partitionEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *partitionEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/* create tuple descriptor for return value */
|
||||
TypeFuncClass resultTypeClass = get_call_result_type(fcinfo, NULL,
|
||||
|
|
|
@ -558,7 +558,7 @@ static List *
|
|||
RelationShardListForShardCreate(ShardInterval *shardInterval)
|
||||
{
|
||||
Oid relationId = shardInterval->relationId;
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *referencedRelationList = cacheEntry->referencedRelationsViaForeignKey;
|
||||
List *referencingRelationList = cacheEntry->referencingRelationsViaForeignKey;
|
||||
int shardIndex = -1;
|
||||
|
|
|
@ -81,7 +81,7 @@ int ReadFromSecondaries = USE_SECONDARY_NODES_NEVER;
|
|||
/*
|
||||
* ShardCacheEntry represents an entry in the shardId -> ShardInterval cache.
|
||||
* To avoid duplicating data and invalidation logic between this cache and the
|
||||
* DistTableCache, this only points into the DistTableCacheEntry of the
|
||||
* DistTableCache, this only points into the CitusTableCacheEntry of the
|
||||
* shard's distributed table.
|
||||
*/
|
||||
typedef struct ShardCacheEntry
|
||||
|
@ -93,7 +93,7 @@ typedef struct ShardCacheEntry
|
|||
* Cache entry for the distributed table a shard belongs to, possibly not
|
||||
* valid.
|
||||
*/
|
||||
DistTableCacheEntry *tableEntry;
|
||||
CitusTableCacheEntry *tableEntry;
|
||||
|
||||
/*
|
||||
* Offset in tableEntry->sortedShardIntervalArray, only valid if
|
||||
|
@ -191,9 +191,9 @@ static ScanKeyData DistObjectScanKey[3];
|
|||
/* local function forward declarations */
|
||||
static bool IsCitusTableViaCatalog(Oid relationId);
|
||||
static ShardCacheEntry * LookupShardCacheEntry(int64 shardId);
|
||||
static DistTableCacheEntry * LookupDistTableCacheEntry(Oid relationId);
|
||||
static void BuildDistTableCacheEntry(DistTableCacheEntry *cacheEntry);
|
||||
static void BuildCachedShardList(DistTableCacheEntry *cacheEntry);
|
||||
static CitusTableCacheEntry * LookupLookupCitusTableCacheEntry(Oid relationId);
|
||||
static void BuildLookupCitusTableCacheEntry(CitusTableCacheEntry *cacheEntry);
|
||||
static void BuildCachedShardList(CitusTableCacheEntry *cacheEntry);
|
||||
static void PrepareWorkerNodeCache(void);
|
||||
static bool CheckInstalledVersion(int elevel);
|
||||
static char * AvailableExtensionVersion(void);
|
||||
|
@ -207,7 +207,7 @@ static void RegisterForeignKeyGraphCacheCallbacks(void);
|
|||
static void RegisterWorkerNodeCacheCallbacks(void);
|
||||
static void RegisterLocalGroupIdCacheCallbacks(void);
|
||||
static uint32 WorkerNodeHashCode(const void *key, Size keySize);
|
||||
static void ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry);
|
||||
static void ResetLookupCitusTableCacheEntry(CitusTableCacheEntry *cacheEntry);
|
||||
static void CreateDistTableCache(void);
|
||||
static void CreateDistObjectCache(void);
|
||||
static void InvalidateForeignRelationGraphCacheCallback(Datum argument, Oid relationId);
|
||||
|
@ -274,11 +274,11 @@ EnsureModificationsCanRun(void)
|
|||
bool
|
||||
IsCitusTable(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = LookupDistTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupLookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/*
|
||||
* If extension hasn't been created, or has the wrong version and the
|
||||
* table isn't a distributed one, LookupDistTableCacheEntry() will return NULL.
|
||||
* table isn't a distributed one, LookupLookupCitusTableCacheEntry() will return NULL.
|
||||
*/
|
||||
if (!cacheEntry)
|
||||
{
|
||||
|
@ -342,7 +342,7 @@ CitusTableList(void)
|
|||
Oid relationId = InvalidOid;
|
||||
foreach_oid(relationId, distTableOidList)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
Assert(cacheEntry->isCitusTable);
|
||||
|
||||
distributedTableList = lappend(distributedTableList, cacheEntry);
|
||||
|
@ -363,7 +363,7 @@ LoadShardInterval(uint64 shardId)
|
|||
{
|
||||
ShardCacheEntry *shardEntry = LookupShardCacheEntry(shardId);
|
||||
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
|
||||
Assert(tableEntry->isCitusTable);
|
||||
|
||||
|
@ -390,7 +390,7 @@ RelationIdForShard(uint64 shardId)
|
|||
{
|
||||
ShardCacheEntry *shardEntry = LookupShardCacheEntry(shardId);
|
||||
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
|
||||
Assert(tableEntry->isCitusTable);
|
||||
|
||||
|
@ -406,7 +406,7 @@ bool
|
|||
ReferenceTableShardId(uint64 shardId)
|
||||
{
|
||||
ShardCacheEntry *shardEntry = LookupShardCacheEntry(shardId);
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
|
||||
return (tableEntry->partitionMethod == DISTRIBUTE_BY_NONE);
|
||||
}
|
||||
|
@ -422,7 +422,7 @@ GroupShardPlacement *
|
|||
LoadGroupShardPlacement(uint64 shardId, uint64 placementId)
|
||||
{
|
||||
ShardCacheEntry *shardEntry = LookupShardCacheEntry(shardId);
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
|
||||
/* the offset better be in a valid range */
|
||||
Assert(shardEntry->shardIndex < tableEntry->shardIntervalArrayLength);
|
||||
|
@ -475,7 +475,7 @@ FindShardPlacementOnGroup(int32 groupId, uint64 shardId)
|
|||
ShardPlacement *placementOnNode = NULL;
|
||||
|
||||
ShardCacheEntry *shardEntry = LookupShardCacheEntry(shardId);
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
GroupShardPlacement *placementArray =
|
||||
tableEntry->arrayOfPlacementArrays[shardEntry->shardIndex];
|
||||
int numberOfPlacements =
|
||||
|
@ -504,7 +504,7 @@ static ShardPlacement *
|
|||
ResolveGroupShardPlacement(GroupShardPlacement *groupShardPlacement,
|
||||
ShardCacheEntry *shardEntry)
|
||||
{
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
int shardIndex = shardEntry->shardIndex;
|
||||
ShardInterval *shardInterval = tableEntry->sortedShardIntervalArray[shardIndex];
|
||||
|
||||
|
@ -662,7 +662,7 @@ ShardPlacementList(uint64 shardId)
|
|||
List *placementList = NIL;
|
||||
|
||||
ShardCacheEntry *shardEntry = LookupShardCacheEntry(shardId);
|
||||
DistTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
CitusTableCacheEntry *tableEntry = shardEntry->tableEntry;
|
||||
|
||||
/* the offset better be in a valid range */
|
||||
Assert(shardEntry->shardIndex < tableEntry->shardIntervalArrayLength);
|
||||
|
@ -722,7 +722,7 @@ LookupShardCacheEntry(int64 shardId)
|
|||
Oid relationId = LookupShardRelation(shardId, false);
|
||||
|
||||
/* trigger building the cache for the shard id */
|
||||
LookupDistTableCacheEntry(relationId);
|
||||
LookupLookupCitusTableCacheEntry(relationId);
|
||||
|
||||
recheck = true;
|
||||
}
|
||||
|
@ -745,8 +745,8 @@ LookupShardCacheEntry(int64 shardId)
|
|||
* reused. Reload the cache entries for both old and new relation
|
||||
* ID and then look up the shard entry again.
|
||||
*/
|
||||
LookupDistTableCacheEntry(oldRelationId);
|
||||
LookupDistTableCacheEntry(currentRelationId);
|
||||
LookupLookupCitusTableCacheEntry(oldRelationId);
|
||||
LookupLookupCitusTableCacheEntry(currentRelationId);
|
||||
|
||||
recheck = true;
|
||||
}
|
||||
|
@ -778,11 +778,11 @@ LookupShardCacheEntry(int64 shardId)
|
|||
*
|
||||
* Errors out if no relation matching the criteria could be found.
|
||||
*/
|
||||
DistTableCacheEntry *
|
||||
CitusTableCacheEntry(Oid distributedRelationId)
|
||||
CitusTableCacheEntry *
|
||||
LookupCitusTableCacheEntry(Oid distributedRelationId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry =
|
||||
LookupDistTableCacheEntry(distributedRelationId);
|
||||
CitusTableCacheEntry *cacheEntry =
|
||||
LookupLookupCitusTableCacheEntry(distributedRelationId);
|
||||
|
||||
if (cacheEntry && cacheEntry->isCitusTable)
|
||||
{
|
||||
|
@ -806,11 +806,11 @@ CitusTableCacheEntry(Oid distributedRelationId)
|
|||
|
||||
|
||||
/*
|
||||
* LookupDistTableCacheEntry returns the distributed table metadata for the
|
||||
* LookupCitusTableCacheEntry returns the distributed table metadata for the
|
||||
* passed relationId. For efficiency it caches lookups.
|
||||
*/
|
||||
static DistTableCacheEntry *
|
||||
LookupDistTableCacheEntry(Oid relationId)
|
||||
static CitusTableCacheEntry *
|
||||
LookupLookupCitusTableCacheEntry(Oid relationId)
|
||||
{
|
||||
bool foundInCache = false;
|
||||
void *hashKey = (void *) &relationId;
|
||||
|
@ -854,8 +854,9 @@ LookupDistTableCacheEntry(Oid relationId)
|
|||
}
|
||||
}
|
||||
|
||||
DistTableCacheEntry *cacheEntry = hash_search(DistTableCacheHash, hashKey, HASH_ENTER,
|
||||
&foundInCache);
|
||||
CitusTableCacheEntry *cacheEntry = hash_search(DistTableCacheHash, hashKey,
|
||||
HASH_ENTER,
|
||||
&foundInCache);
|
||||
|
||||
/* return valid matches */
|
||||
if (foundInCache)
|
||||
|
@ -872,12 +873,12 @@ LookupDistTableCacheEntry(Oid relationId)
|
|||
}
|
||||
|
||||
/* free the content of old, invalid, entries */
|
||||
ResetDistTableCacheEntry(cacheEntry);
|
||||
ResetLookupCitusTableCacheEntry(cacheEntry);
|
||||
}
|
||||
|
||||
/* zero out entry, but not the key part */
|
||||
memset(((char *) cacheEntry) + sizeof(Oid), 0,
|
||||
sizeof(DistTableCacheEntry) - sizeof(Oid));
|
||||
sizeof(CitusTableCacheEntry) - sizeof(Oid));
|
||||
|
||||
/*
|
||||
* We disable interrupts while creating the cache entry because loading
|
||||
|
@ -888,7 +889,7 @@ LookupDistTableCacheEntry(Oid relationId)
|
|||
HOLD_INTERRUPTS();
|
||||
|
||||
/* actually fill out entry */
|
||||
BuildDistTableCacheEntry(cacheEntry);
|
||||
BuildLookupCitusTableCacheEntry(cacheEntry);
|
||||
|
||||
/* and finally mark as valid */
|
||||
cacheEntry->isValid = true;
|
||||
|
@ -1002,11 +1003,11 @@ LookupDistObjectCacheEntry(Oid classid, Oid objid, int32 objsubid)
|
|||
|
||||
|
||||
/*
|
||||
* BuildDistTableCacheEntry is a helper routine for
|
||||
* LookupDistTableCacheEntry() for building the cache contents.
|
||||
* BuildCitusTableCacheEntry is a helper routine for
|
||||
* LookupLookupCitusTableCacheEntry() for building the cache contents.
|
||||
*/
|
||||
static void
|
||||
BuildDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
|
||||
BuildLookupCitusTableCacheEntry(CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
MemoryContext oldContext = NULL;
|
||||
Datum datumArray[Natts_pg_dist_partition];
|
||||
|
@ -1118,11 +1119,11 @@ BuildDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
|
|||
|
||||
|
||||
/*
|
||||
* BuildCachedShardList() is a helper routine for BuildDistTableCacheEntry()
|
||||
* BuildCachedShardList() is a helper routine for BuildLookupCitusTableCacheEntry()
|
||||
* building up the list of shards in a distributed relation.
|
||||
*/
|
||||
static void
|
||||
BuildCachedShardList(DistTableCacheEntry *cacheEntry)
|
||||
BuildCachedShardList(CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
ShardInterval **shardIntervalArray = NULL;
|
||||
ShardInterval **sortedShardIntervalArray = NULL;
|
||||
|
@ -1200,7 +1201,7 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry)
|
|||
* and we can safely wipe its entry, which will remove all corresponding
|
||||
* shard cache entries.
|
||||
*/
|
||||
ResetDistTableCacheEntry(shardEntry->tableEntry);
|
||||
ResetLookupCitusTableCacheEntry(shardEntry->tableEntry);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1287,7 +1288,7 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry)
|
|||
}
|
||||
|
||||
/*
|
||||
* We set these here, so ResetDistTableCacheEntry() can see what has been
|
||||
* We set these here, so ResetLookupCitusTableCacheEntry() can see what has been
|
||||
* entered into DistShardCacheHash even if the following loop is interrupted
|
||||
* by throwing errors, etc.
|
||||
*/
|
||||
|
@ -1314,7 +1315,7 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry)
|
|||
|
||||
/*
|
||||
* We should increment this only after we are sure this hasn't already
|
||||
* been assigned to any other relations. ResetDistTableCacheEntry()
|
||||
* been assigned to any other relations. ResetLookupCitusTableCacheEntry()
|
||||
* depends on this.
|
||||
*/
|
||||
cacheEntry->shardIntervalArrayLength++;
|
||||
|
@ -1355,7 +1356,7 @@ BuildCachedShardList(DistTableCacheEntry *cacheEntry)
|
|||
* our expectations.
|
||||
*/
|
||||
void
|
||||
ErrorIfInconsistentShardIntervals(DistTableCacheEntry *cacheEntry)
|
||||
ErrorIfInconsistentShardIntervals(CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
/*
|
||||
* If table is hash-partitioned and has shards, there never should be any
|
||||
|
@ -3241,11 +3242,11 @@ WorkerNodeHashCode(const void *key, Size keySize)
|
|||
|
||||
|
||||
/*
|
||||
* ResetDistTableCacheEntry frees any out-of-band memory used by a cache entry,
|
||||
* ResetCitusTableCacheEntry frees any out-of-band memory used by a cache entry,
|
||||
* but does not free the entry itself.
|
||||
*/
|
||||
static void
|
||||
ResetDistTableCacheEntry(DistTableCacheEntry *cacheEntry)
|
||||
ResetLookupCitusTableCacheEntry(CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
if (cacheEntry->partitionKeyString != NULL)
|
||||
{
|
||||
|
@ -3405,8 +3406,8 @@ InvalidateDistRelationCacheCallback(Datum argument, Oid relationId)
|
|||
bool foundInCache = false;
|
||||
|
||||
|
||||
DistTableCacheEntry *cacheEntry = hash_search(DistTableCacheHash, hashKey,
|
||||
HASH_FIND, &foundInCache);
|
||||
CitusTableCacheEntry *cacheEntry = hash_search(DistTableCacheHash, hashKey,
|
||||
HASH_FIND, &foundInCache);
|
||||
if (foundInCache)
|
||||
{
|
||||
cacheEntry->isValid = false;
|
||||
|
@ -3436,12 +3437,12 @@ InvalidateDistRelationCacheCallback(Datum argument, Oid relationId)
|
|||
static void
|
||||
InvalidateDistTableCache(void)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = NULL;
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
HASH_SEQ_STATUS status;
|
||||
|
||||
hash_seq_init(&status, DistTableCacheHash);
|
||||
|
||||
while ((cacheEntry = (DistTableCacheEntry *) hash_seq_search(&status)) != NULL)
|
||||
while ((cacheEntry = (CitusTableCacheEntry *) hash_seq_search(&status)) != NULL)
|
||||
{
|
||||
cacheEntry->isValid = false;
|
||||
}
|
||||
|
@ -3473,14 +3474,14 @@ InvalidateDistObjectCache(void)
|
|||
void
|
||||
FlushDistTableCache(void)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = NULL;
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
HASH_SEQ_STATUS status;
|
||||
|
||||
hash_seq_init(&status, DistTableCacheHash);
|
||||
|
||||
while ((cacheEntry = (DistTableCacheEntry *) hash_seq_search(&status)) != NULL)
|
||||
while ((cacheEntry = (CitusTableCacheEntry *) hash_seq_search(&status)) != NULL)
|
||||
{
|
||||
ResetDistTableCacheEntry(cacheEntry);
|
||||
ResetLookupCitusTableCacheEntry(cacheEntry);
|
||||
}
|
||||
|
||||
hash_destroy(DistTableCacheHash);
|
||||
|
@ -3495,7 +3496,7 @@ CreateDistTableCache(void)
|
|||
HASHCTL info;
|
||||
MemSet(&info, 0, sizeof(info));
|
||||
info.keysize = sizeof(Oid);
|
||||
info.entrysize = sizeof(DistTableCacheEntry);
|
||||
info.entrysize = sizeof(CitusTableCacheEntry);
|
||||
info.hash = tag_hash;
|
||||
info.hcxt = MetadataCacheMemoryContext;
|
||||
DistTableCacheHash =
|
||||
|
|
|
@ -222,7 +222,7 @@ ClusterHasKnownMetadataWorkers()
|
|||
bool
|
||||
ShouldSyncTableMetadata(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *tableEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *tableEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
bool hashDistributed = (tableEntry->partitionMethod == DISTRIBUTE_BY_HASH);
|
||||
bool streamingReplicated =
|
||||
|
@ -371,7 +371,7 @@ MetadataCreateCommands(void)
|
|||
nodeListInsertCommand);
|
||||
|
||||
/* create the list of tables whose metadata will be created */
|
||||
DistTableCacheEntry *cacheEntry = NULL;
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
foreach_ptr(cacheEntry, distributedTableList)
|
||||
{
|
||||
if (ShouldSyncTableMetadata(cacheEntry->relationId))
|
||||
|
@ -466,7 +466,7 @@ MetadataCreateCommands(void)
|
|||
List *
|
||||
GetDistributedTableDDLEvents(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
List *commandList = NIL;
|
||||
bool includeSequenceDefaults = true;
|
||||
|
@ -618,7 +618,7 @@ NodeListInsertCommand(List *workerNodeList)
|
|||
* executed to replicate the metadata for a distributed table.
|
||||
*/
|
||||
char *
|
||||
DistributionCreateCommand(DistTableCacheEntry *cacheEntry)
|
||||
DistributionCreateCommand(CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
StringInfo insertDistributionCommand = makeStringInfo();
|
||||
Oid relationId = cacheEntry->relationId;
|
||||
|
@ -1308,7 +1308,7 @@ DetachPartitionCommandList(void)
|
|||
List *distributedTableList = CitusTableList();
|
||||
|
||||
/* we iterate over all distributed partitioned tables and DETACH their partitions */
|
||||
DistTableCacheEntry *cacheEntry = NULL;
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
foreach_ptr(cacheEntry, distributedTableList)
|
||||
{
|
||||
if (!PartitionedTable(cacheEntry->relationId))
|
||||
|
|
|
@ -813,7 +813,7 @@ get_shard_id_for_distribution_column(PG_FUNCTION_ARGS)
|
|||
else if (distributionMethod == DISTRIBUTE_BY_HASH ||
|
||||
distributionMethod == DISTRIBUTE_BY_RANGE)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/* if given table is not reference table, distributionValue cannot be NULL */
|
||||
if (PG_ARGISNULL(1))
|
||||
|
|
|
@ -1833,7 +1833,7 @@ void
|
|||
multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
|
||||
Index restrictionIndex, RangeTblEntry *rte)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = NULL;
|
||||
CitusTableCacheEntry *cacheEntry = NULL;
|
||||
|
||||
if (ReplaceCitusExtraDataContainer && IsCitusExtraDataContainerRelation(rte))
|
||||
{
|
||||
|
@ -1897,7 +1897,7 @@ multi_relation_restriction_hook(PlannerInfo *root, RelOptInfo *relOptInfo,
|
|||
*/
|
||||
if (distributedTable)
|
||||
{
|
||||
cacheEntry = CitusTableCacheEntry(rte->relid);
|
||||
cacheEntry = LookupCitusTableCacheEntry(rte->relid);
|
||||
|
||||
relationRestrictionContext->allReferenceTables &=
|
||||
(cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE);
|
||||
|
@ -2426,7 +2426,8 @@ IsLocalReferenceTableJoin(Query *parse, List *rangeTableList)
|
|||
continue;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(rangeTableEntry->relid);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(
|
||||
rangeTableEntry->relid);
|
||||
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
hasReferenceTable = true;
|
||||
|
@ -2496,7 +2497,7 @@ UpdateReferenceTablesWithShard(Node *node, void *context)
|
|||
return false;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
if (cacheEntry->partitionMethod != DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
return false;
|
||||
|
|
|
@ -202,7 +202,7 @@ FastPathRouterQuery(Query *query, Node **distributionKeyValue)
|
|||
|
||||
/* we don't want to deal with append/range distributed tables */
|
||||
Oid distributedTableId = rangeTableEntry->relid;
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_HASH ||
|
||||
cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE))
|
||||
{
|
||||
|
|
|
@ -108,7 +108,7 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
|
|||
Datum partitionValueDatum = 0;
|
||||
ShardInterval *shardInterval = NULL;
|
||||
List *placementList = NIL;
|
||||
DistTableCacheEntry *distTable = NULL;
|
||||
CitusTableCacheEntry *distTable = NULL;
|
||||
Var *partitionColumn = NULL;
|
||||
ShardPlacement *placement = NULL;
|
||||
WorkerNode *workerNode = NULL;
|
||||
|
@ -267,7 +267,7 @@ TryToDelegateFunctionCall(DistributedPlanningContext *planContext)
|
|||
return NULL;
|
||||
}
|
||||
|
||||
distTable = CitusTableCacheEntry(colocatedRelationId);
|
||||
distTable = LookupCitusTableCacheEntry(colocatedRelationId);
|
||||
partitionColumn = distTable->partitionColumn;
|
||||
if (partitionColumn == NULL)
|
||||
{
|
||||
|
|
|
@ -218,7 +218,7 @@ CreateDistributedInsertSelectPlan(Query *originalQuery,
|
|||
RangeTblEntry *insertRte = ExtractResultRelationRTE(originalQuery);
|
||||
RangeTblEntry *subqueryRte = ExtractSelectRangeTableEntry(originalQuery);
|
||||
Oid targetRelationId = insertRte->relid;
|
||||
DistTableCacheEntry *targetCacheEntry = CitusTableCacheEntry(targetRelationId);
|
||||
CitusTableCacheEntry *targetCacheEntry = LookupCitusTableCacheEntry(targetRelationId);
|
||||
int shardCount = targetCacheEntry->shardIntervalArrayLength;
|
||||
RelationRestrictionContext *relationRestrictionContext =
|
||||
plannerRestrictionContext->relationRestrictionContext;
|
||||
|
@ -426,7 +426,7 @@ RouterModifyTaskForShardInterval(Query *originalQuery, ShardInterval *shardInter
|
|||
|
||||
uint64 shardId = shardInterval->shardId;
|
||||
Oid distributedTableId = shardInterval->relationId;
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
|
||||
PlannerRestrictionContext *copyOfPlannerRestrictionContext = palloc0(
|
||||
sizeof(PlannerRestrictionContext));
|
||||
|
|
|
@ -1380,7 +1380,7 @@ PartitionColumn(Oid relationId, uint32 rangeTableId)
|
|||
Var *
|
||||
DistPartitionKey(Oid relationId)
|
||||
{
|
||||
DistTableCacheEntry *partitionEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *partitionEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/* reference tables do not have partition column */
|
||||
if (partitionEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
||||
|
@ -1417,7 +1417,7 @@ char
|
|||
PartitionMethod(Oid relationId)
|
||||
{
|
||||
/* errors out if not a distributed table */
|
||||
DistTableCacheEntry *partitionEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *partitionEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
char partitionMethod = partitionEntry->partitionMethod;
|
||||
|
||||
|
@ -1430,7 +1430,7 @@ char
|
|||
TableReplicationModel(Oid relationId)
|
||||
{
|
||||
/* errors out if not a distributed table */
|
||||
DistTableCacheEntry *partitionEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *partitionEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
char replicationModel = partitionEntry->replicationModel;
|
||||
|
||||
|
|
|
@ -1950,7 +1950,7 @@ BuildMapMergeJob(Query *jobQuery, List *dependentJobList, Var *partitionKey,
|
|||
else if (partitionType == SINGLE_HASH_PARTITION_TYPE || partitionType ==
|
||||
RANGE_PARTITION_TYPE)
|
||||
{
|
||||
DistTableCacheEntry *cache = CitusTableCacheEntry(baseRelationId);
|
||||
CitusTableCacheEntry *cache = LookupCitusTableCacheEntry(baseRelationId);
|
||||
uint32 shardCount = cache->shardIntervalArrayLength;
|
||||
ShardInterval **sortedShardIntervalArray = cache->sortedShardIntervalArray;
|
||||
|
||||
|
@ -2177,7 +2177,7 @@ QueryPushdownSqlTaskList(Query *query, uint64 jobId,
|
|||
List *prunedShardList = (List *) lfirst(prunedRelationShardCell);
|
||||
ListCell *shardIntervalCell = NULL;
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
continue;
|
||||
|
@ -2309,7 +2309,7 @@ ErrorIfUnsupportedShardDistribution(Query *query)
|
|||
}
|
||||
else
|
||||
{
|
||||
DistTableCacheEntry *distTableEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *distTableEntry = LookupCitusTableCacheEntry(relationId);
|
||||
if (distTableEntry->hasOverlappingShardInterval)
|
||||
{
|
||||
ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
|
||||
|
@ -2415,7 +2415,7 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
|
|||
Oid relationId = relationRestriction->relationId;
|
||||
ShardInterval *shardInterval = NULL;
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
/* reference table only has one shard */
|
||||
|
@ -2509,8 +2509,8 @@ QueryPushdownTaskCreate(Query *originalQuery, int shardIndex,
|
|||
bool
|
||||
CoPartitionedTables(Oid firstRelationId, Oid secondRelationId)
|
||||
{
|
||||
DistTableCacheEntry *firstTableCache = CitusTableCacheEntry(firstRelationId);
|
||||
DistTableCacheEntry *secondTableCache = CitusTableCacheEntry(secondRelationId);
|
||||
CitusTableCacheEntry *firstTableCache = LookupCitusTableCacheEntry(firstRelationId);
|
||||
CitusTableCacheEntry *secondTableCache = LookupCitusTableCacheEntry(secondRelationId);
|
||||
|
||||
ShardInterval **sortedFirstIntervalArray = firstTableCache->sortedShardIntervalArray;
|
||||
ShardInterval **sortedSecondIntervalArray =
|
||||
|
@ -3903,8 +3903,8 @@ FragmentInterval(RangeTableFragment *fragment)
|
|||
bool
|
||||
ShardIntervalsOverlap(ShardInterval *firstInterval, ShardInterval *secondInterval)
|
||||
{
|
||||
DistTableCacheEntry *intervalRelation =
|
||||
CitusTableCacheEntry(firstInterval->relationId);
|
||||
CitusTableCacheEntry *intervalRelation =
|
||||
LookupCitusTableCacheEntry(firstInterval->relationId);
|
||||
|
||||
Assert(intervalRelation->partitionMethod != DISTRIBUTE_BY_NONE);
|
||||
|
||||
|
|
|
@ -135,7 +135,7 @@ static bool TargetEntryChangesValue(TargetEntry *targetEntry, Var *column,
|
|||
FromExpr *joinTree);
|
||||
static Job * RouterInsertJob(Query *originalQuery, Query *query,
|
||||
DeferredErrorMessage **planningError);
|
||||
static void ErrorIfNoShardsExist(DistTableCacheEntry *cacheEntry);
|
||||
static void ErrorIfNoShardsExist(CitusTableCacheEntry *cacheEntry);
|
||||
static DeferredErrorMessage * DeferErrorIfModifyView(Query *queryTree);
|
||||
static bool CanShardPrune(Oid distributedTableId, Query *query);
|
||||
static Job * CreateJob(Query *query);
|
||||
|
@ -741,7 +741,7 @@ ModifyQuerySupported(Query *queryTree, Query *originalQuery, bool multiShardQuer
|
|||
if (rangeTableEntry->rtekind == RTE_SUBQUERY)
|
||||
{
|
||||
StringInfo errorHint = makeStringInfo();
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(
|
||||
distributedTableId);
|
||||
char *partitionKeyString = cacheEntry->partitionKeyString;
|
||||
char *partitionColumnName = ColumnToColumnName(distributedTableId,
|
||||
|
@ -1541,7 +1541,7 @@ CanShardPrune(Oid distributedTableId, Query *query)
|
|||
* ErrorIfNoShardsExist throws an error if the given table has no shards.
|
||||
*/
|
||||
static void
|
||||
ErrorIfNoShardsExist(DistTableCacheEntry *cacheEntry)
|
||||
ErrorIfNoShardsExist(CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
int shardCount = cacheEntry->shardIntervalArrayLength;
|
||||
if (shardCount == 0)
|
||||
|
@ -1571,7 +1571,7 @@ RouterInsertTaskList(Query *query, bool parametersInQueryResolved,
|
|||
ListCell *modifyRouteCell = NULL;
|
||||
|
||||
Oid distributedTableId = ExtractFirstCitusTableId(query);
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
|
||||
ErrorIfNoShardsExist(cacheEntry);
|
||||
|
||||
|
@ -1978,7 +1978,7 @@ SingleShardModifyTaskList(Query *query, uint64 jobId, List *relationShardList,
|
|||
RangeTblEntry *updateOrDeleteRTE = GetUpdateOrDeleteRTE(query);
|
||||
Assert(updateOrDeleteRTE != NULL);
|
||||
|
||||
DistTableCacheEntry *modificationTableCacheEntry = CitusTableCacheEntry(
|
||||
CitusTableCacheEntry *modificationTableCacheEntry = LookupCitusTableCacheEntry(
|
||||
updateOrDeleteRTE->relid);
|
||||
char modificationPartitionMethod = modificationTableCacheEntry->partitionMethod;
|
||||
|
||||
|
@ -2043,7 +2043,7 @@ SelectsFromDistributedTable(List *rangeTableList, Query *query)
|
|||
continue;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(
|
||||
rangeTableEntry->relid);
|
||||
if (cacheEntry->partitionMethod != DISTRIBUTE_BY_NONE &&
|
||||
(resultRangeTableEntry == NULL || resultRangeTableEntry->relid !=
|
||||
|
@ -2409,7 +2409,7 @@ TargetShardIntervalForFastPathQuery(Query *query, bool *isMultiShardQuery,
|
|||
|
||||
if (inputDistributionKeyValue && !inputDistributionKeyValue->constisnull)
|
||||
{
|
||||
DistTableCacheEntry *cache = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cache = LookupCitusTableCacheEntry(relationId);
|
||||
ShardInterval *shardInterval =
|
||||
FindShardInterval(inputDistributionKeyValue->constvalue, cache);
|
||||
if (shardInterval == NULL)
|
||||
|
@ -2495,7 +2495,7 @@ TargetShardIntervalsForRestrictInfo(RelationRestrictionContext *restrictionConte
|
|||
(RelationRestriction *) lfirst(restrictionCell);
|
||||
Oid relationId = relationRestriction->relationId;
|
||||
Index tableId = relationRestriction->index;
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
int shardCount = cacheEntry->shardIntervalArrayLength;
|
||||
List *baseRestrictionList = relationRestriction->relOptInfo->baserestrictinfo;
|
||||
List *restrictClauseList = get_all_actual_clauses(baseRestrictionList);
|
||||
|
@ -2660,7 +2660,7 @@ static List *
|
|||
BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
|
||||
{
|
||||
Oid distributedTableId = ExtractFirstCitusTableId(query);
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
char partitionMethod = cacheEntry->partitionMethod;
|
||||
uint32 rangeTableId = 1;
|
||||
List *modifyRouteList = NIL;
|
||||
|
@ -2730,7 +2730,7 @@ BuildRoutesForInsert(Query *query, DeferredErrorMessage **planningError)
|
|||
{
|
||||
Datum partitionValue = partitionValueConst->constvalue;
|
||||
|
||||
cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
ShardInterval *shardInterval = FindShardInterval(partitionValue, cacheEntry);
|
||||
if (shardInterval != NULL)
|
||||
{
|
||||
|
|
|
@ -253,12 +253,12 @@ static int PerformValueCompare(FunctionCallInfo compareFunctionCall, Datum a,
|
|||
Datum b);
|
||||
static int PerformCompare(FunctionCallInfo compareFunctionCall);
|
||||
|
||||
static List * PruneOne(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
static List * PruneOne(CitusTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
PruningInstance *prune);
|
||||
static List * PruneWithBoundaries(DistTableCacheEntry *cacheEntry,
|
||||
static List * PruneWithBoundaries(CitusTableCacheEntry *cacheEntry,
|
||||
ClauseWalkerContext *context,
|
||||
PruningInstance *prune);
|
||||
static List * ExhaustivePrune(DistTableCacheEntry *cacheEntry,
|
||||
static List * ExhaustivePrune(CitusTableCacheEntry *cacheEntry,
|
||||
ClauseWalkerContext *context,
|
||||
PruningInstance *prune);
|
||||
static bool ExhaustivePruneOne(ShardInterval *curInterval,
|
||||
|
@ -294,7 +294,7 @@ List *
|
|||
PruneShards(Oid relationId, Index rangeTableId, List *whereClauseList,
|
||||
Const **partitionValueConst)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
int shardCount = cacheEntry->shardIntervalArrayLength;
|
||||
char partitionMethod = cacheEntry->partitionMethod;
|
||||
ClauseWalkerContext context = { 0 };
|
||||
|
@ -438,7 +438,7 @@ PruneShards(Oid relationId, Index rangeTableId, List *whereClauseList,
|
|||
* We can use list_union_ptr, which is a lot faster than doing
|
||||
* comparing shards by value, because all the ShardIntervals are
|
||||
* guaranteed to be from
|
||||
* DistTableCacheEntry->sortedShardIntervalArray (thus having the
|
||||
* CitusTableCacheEntry->sortedShardIntervalArray (thus having the
|
||||
* same pointer values).
|
||||
*/
|
||||
prunedList = list_union_ptr(prunedList, pruneOneList);
|
||||
|
@ -495,7 +495,7 @@ PruneShards(Oid relationId, Index rangeTableId, List *whereClauseList,
|
|||
}
|
||||
|
||||
/*
|
||||
* Deep copy list, so it's independent of the DistTableCacheEntry
|
||||
* Deep copy list, so it's independent of the CitusTableCacheEntry
|
||||
* contents.
|
||||
*/
|
||||
return DeepCopyShardIntervalList(prunedList);
|
||||
|
@ -1360,7 +1360,7 @@ DeepCopyShardIntervalList(List *originalShardIntervalList)
|
|||
* PruningInstance.
|
||||
*/
|
||||
static List *
|
||||
PruneOne(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
PruneOne(CitusTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
PruningInstance *prune)
|
||||
{
|
||||
ShardInterval *shardInterval = NULL;
|
||||
|
@ -1667,7 +1667,7 @@ UpperShardBoundary(Datum partitionColumnValue, ShardInterval **shardIntervalCach
|
|||
* list of surviving shards.
|
||||
*/
|
||||
static List *
|
||||
PruneWithBoundaries(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
PruneWithBoundaries(CitusTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
PruningInstance *prune)
|
||||
{
|
||||
List *remainingShardList = NIL;
|
||||
|
@ -1782,7 +1782,7 @@ PruneWithBoundaries(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *contex
|
|||
* constraints, by simply checking them for each individual shard.
|
||||
*/
|
||||
static List *
|
||||
ExhaustivePrune(DistTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
ExhaustivePrune(CitusTableCacheEntry *cacheEntry, ClauseWalkerContext *context,
|
||||
PruningInstance *prune)
|
||||
{
|
||||
List *remainingShardList = NIL;
|
||||
|
|
|
@ -64,7 +64,7 @@ partition_task_list_results(PG_FUNCTION_ARGS)
|
|||
Job *job = distributedPlan->workerJob;
|
||||
List *taskList = job->taskList;
|
||||
|
||||
DistTableCacheEntry *targetRelation = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *targetRelation = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/*
|
||||
* Here SELECT query's target list should match column list of target relation,
|
||||
|
@ -132,7 +132,7 @@ redistribute_task_list_results(PG_FUNCTION_ARGS)
|
|||
Job *job = distributedPlan->workerJob;
|
||||
List *taskList = job->taskList;
|
||||
|
||||
DistTableCacheEntry *targetRelation = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *targetRelation = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/*
|
||||
* Here SELECT query's target list should match column list of target relation,
|
||||
|
|
|
@ -38,7 +38,7 @@ get_referencing_relation_id_list(PG_FUNCTION_ARGS)
|
|||
if (SRF_IS_FIRSTCALL())
|
||||
{
|
||||
Oid relationId = PG_GETARG_OID(0);
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *refList = cacheEntry->referencingRelationsViaForeignKey;
|
||||
|
||||
/* create a function context for cross-call persistence */
|
||||
|
@ -89,7 +89,7 @@ get_referenced_relation_id_list(PG_FUNCTION_ARGS)
|
|||
if (SRF_IS_FIRSTCALL())
|
||||
{
|
||||
Oid relationId = PG_GETARG_OID(0);
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *refList = cacheEntry->referencedRelationsViaForeignKey;
|
||||
|
||||
/* create a function context for cross-call persistence */
|
||||
|
|
|
@ -241,7 +241,7 @@ SortedShardIntervalArray(Oid distributedTableId)
|
|||
{
|
||||
Oid shardIdTypeId = INT8OID;
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
ShardInterval **shardIntervalArray = cacheEntry->sortedShardIntervalArray;
|
||||
int shardIdCount = cacheEntry->shardIntervalArrayLength;
|
||||
Datum *shardIdDatumArray = palloc0(shardIdCount * sizeof(Datum));
|
||||
|
|
|
@ -690,7 +690,7 @@ CheckConflictingRelationAccesses(Oid relationId, ShardPlacementAccessType access
|
|||
return;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE &&
|
||||
cacheEntry->referencingRelationsViaForeignKey != NIL))
|
||||
|
@ -811,7 +811,7 @@ CheckConflictingParallelRelationAccesses(Oid relationId, ShardPlacementAccessTyp
|
|||
return;
|
||||
}
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
if (!(cacheEntry->partitionMethod == DISTRIBUTE_BY_HASH &&
|
||||
cacheEntry->referencedRelationsViaForeignKey != NIL))
|
||||
{
|
||||
|
@ -882,7 +882,7 @@ HoldsConflictingLockWithReferencedRelations(Oid relationId, ShardPlacementAccess
|
|||
ShardPlacementAccessType *
|
||||
conflictingAccessMode)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
Oid referencedRelation = InvalidOid;
|
||||
foreach_oid(referencedRelation, cacheEntry->referencedRelationsViaForeignKey)
|
||||
|
@ -947,7 +947,7 @@ HoldsConflictingLockWithReferencingRelations(Oid relationId, ShardPlacementAcces
|
|||
ShardPlacementAccessType *
|
||||
conflictingAccessMode)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
bool holdsConflictingLocks = false;
|
||||
|
||||
Assert(PartitionMethod(relationId) == DISTRIBUTE_BY_NONE);
|
||||
|
|
|
@ -546,10 +546,10 @@ GetNextColocationId()
|
|||
void
|
||||
CheckReplicationModel(Oid sourceRelationId, Oid targetRelationId)
|
||||
{
|
||||
DistTableCacheEntry *sourceTableEntry = CitusTableCacheEntry(sourceRelationId);
|
||||
CitusTableCacheEntry *sourceTableEntry = LookupCitusTableCacheEntry(sourceRelationId);
|
||||
char sourceReplicationModel = sourceTableEntry->replicationModel;
|
||||
|
||||
DistTableCacheEntry *targetTableEntry = CitusTableCacheEntry(targetRelationId);
|
||||
CitusTableCacheEntry *targetTableEntry = LookupCitusTableCacheEntry(targetRelationId);
|
||||
char targetReplicationModel = targetTableEntry->replicationModel;
|
||||
|
||||
if (sourceReplicationModel != targetReplicationModel)
|
||||
|
@ -690,7 +690,7 @@ UpdateRelationColocationGroup(Oid distributedRelationId, uint32 colocationId)
|
|||
uint32
|
||||
TableColocationId(Oid distributedTableId)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
|
||||
return cacheEntry->colocationId;
|
||||
}
|
||||
|
@ -836,7 +836,7 @@ ColocatedShardIntervalList(ShardInterval *shardInterval)
|
|||
Oid distributedTableId = shardInterval->relationId;
|
||||
List *colocatedShardList = NIL;
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
char partitionMethod = cacheEntry->partitionMethod;
|
||||
|
||||
/*
|
||||
|
@ -863,8 +863,8 @@ ColocatedShardIntervalList(ShardInterval *shardInterval)
|
|||
Oid colocatedTableId = InvalidOid;
|
||||
foreach_oid(colocatedTableId, colocatedTableList)
|
||||
{
|
||||
DistTableCacheEntry *colocatedTableCacheEntry =
|
||||
CitusTableCacheEntry(colocatedTableId);
|
||||
CitusTableCacheEntry *colocatedTableCacheEntry =
|
||||
LookupCitusTableCacheEntry(colocatedTableId);
|
||||
|
||||
/*
|
||||
* Since we iterate over co-located tables, shard count of each table should be
|
||||
|
@ -966,7 +966,7 @@ ColocatedTableId(Oid colocationId)
|
|||
uint64
|
||||
ColocatedShardIdInRelation(Oid relationId, int shardIndex)
|
||||
{
|
||||
DistTableCacheEntry *tableCacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *tableCacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
return tableCacheEntry->sortedShardIntervalArray[shardIndex]->shardId;
|
||||
}
|
||||
|
|
|
@ -70,7 +70,7 @@ upgrade_to_reference_table(PG_FUNCTION_ARGS)
|
|||
"create_reference_table('%s');", relationName)));
|
||||
}
|
||||
|
||||
DistTableCacheEntry *tableEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *tableEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
if (tableEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
|
@ -463,7 +463,7 @@ ReferenceTableOidList()
|
|||
Oid relationId = InvalidOid;
|
||||
foreach_oid(relationId, distTableOidList)
|
||||
{
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
if (cacheEntry->partitionMethod == DISTRIBUTE_BY_NONE)
|
||||
{
|
||||
|
|
|
@ -361,7 +361,7 @@ SetLocktagForShardDistributionMetadata(int64 shardId, LOCKTAG *tag)
|
|||
{
|
||||
ShardInterval *shardInterval = LoadShardInterval(shardId);
|
||||
Oid citusTableId = shardInterval->relationId;
|
||||
DistTableCacheEntry *citusTable = CitusTableCacheEntry(citusTableId);
|
||||
CitusTableCacheEntry *citusTable = LookupCitusTableCacheEntry(citusTableId);
|
||||
uint32 colocationId = citusTable->colocationId;
|
||||
|
||||
if (colocationId == INVALID_COLOCATION_ID ||
|
||||
|
@ -390,7 +390,7 @@ LockReferencedReferenceShardDistributionMetadata(uint64 shardId, LOCKMODE lockMo
|
|||
{
|
||||
Oid relationId = RelationIdForShard(shardId);
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
List *referencedRelationList = cacheEntry->referencedRelationsViaForeignKey;
|
||||
List *shardIntervalList = GetSortedReferenceShardIntervals(referencedRelationList);
|
||||
|
||||
|
@ -420,7 +420,7 @@ LockReferencedReferenceShardResources(uint64 shardId, LOCKMODE lockMode)
|
|||
{
|
||||
Oid relationId = RelationIdForShard(shardId);
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(relationId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(relationId);
|
||||
|
||||
/*
|
||||
* Note that referencedRelationsViaForeignKey contains transitively referenced
|
||||
|
|
|
@ -239,7 +239,7 @@ ShardIndex(ShardInterval *shardInterval)
|
|||
Oid distributedTableId = shardInterval->relationId;
|
||||
Datum shardMinValue = shardInterval->minValue;
|
||||
|
||||
DistTableCacheEntry *cacheEntry = CitusTableCacheEntry(distributedTableId);
|
||||
CitusTableCacheEntry *cacheEntry = LookupCitusTableCacheEntry(distributedTableId);
|
||||
char partitionMethod = cacheEntry->partitionMethod;
|
||||
|
||||
/*
|
||||
|
@ -275,7 +275,7 @@ ShardIndex(ShardInterval *shardInterval)
|
|||
* as NULL for them.
|
||||
*/
|
||||
ShardInterval *
|
||||
FindShardInterval(Datum partitionColumnValue, DistTableCacheEntry *cacheEntry)
|
||||
FindShardInterval(Datum partitionColumnValue, CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
Datum searchedValue = partitionColumnValue;
|
||||
|
||||
|
@ -310,7 +310,7 @@ FindShardInterval(Datum partitionColumnValue, DistTableCacheEntry *cacheEntry)
|
|||
* of [INT32_MIN, INT32_MAX] can fire this.
|
||||
*/
|
||||
int
|
||||
FindShardIntervalIndex(Datum searchedValue, DistTableCacheEntry *cacheEntry)
|
||||
FindShardIntervalIndex(Datum searchedValue, CitusTableCacheEntry *cacheEntry)
|
||||
{
|
||||
ShardInterval **shardIntervalCache = cacheEntry->sortedShardIntervalArray;
|
||||
int shardCount = cacheEntry->shardIntervalArrayLength;
|
||||
|
|
|
@ -89,7 +89,7 @@ typedef struct CitusCopyDestReceiver
|
|||
int partitionColumnIndex;
|
||||
|
||||
/* distributed table metadata */
|
||||
DistTableCacheEntry *tableMetadata;
|
||||
CitusTableCacheEntry *tableMetadata;
|
||||
|
||||
/* open relation handle */
|
||||
Relation distributedRelation;
|
||||
|
|
|
@ -64,11 +64,11 @@ extern char * CreateIntermediateResultsDirectory(void);
|
|||
extern List ** RedistributeTaskListResults(const char *resultIdPrefix,
|
||||
List *selectTaskList,
|
||||
int partitionColumnIndex,
|
||||
DistTableCacheEntry *targetRelation,
|
||||
CitusTableCacheEntry *targetRelation,
|
||||
bool binaryFormat);
|
||||
extern List * PartitionTasklistResults(const char *resultIdPrefix, List *selectTaskList,
|
||||
int partitionColumnIndex,
|
||||
DistTableCacheEntry *distributionScheme,
|
||||
CitusTableCacheEntry *distributionScheme,
|
||||
bool binaryFormat);
|
||||
|
||||
#endif /* INTERMEDIATE_RESULTS_H */
|
||||
|
|
|
@ -95,7 +95,7 @@ typedef struct
|
|||
/* pg_dist_placement metadata */
|
||||
GroupShardPlacement **arrayOfPlacementArrays;
|
||||
int *arrayOfPlacementArrayLengths;
|
||||
} DistTableCacheEntry;
|
||||
} CitusTableCacheEntry;
|
||||
|
||||
typedef struct DistObjectCacheEntryKey
|
||||
{
|
||||
|
@ -125,7 +125,7 @@ extern bool ReferenceTableShardId(uint64 shardId);
|
|||
extern ShardPlacement * FindShardPlacementOnGroup(int32 groupId, uint64 shardId);
|
||||
extern GroupShardPlacement * LoadGroupShardPlacement(uint64 shardId, uint64 placementId);
|
||||
extern ShardPlacement * LoadShardPlacement(uint64 shardId, uint64 placementId);
|
||||
extern DistTableCacheEntry * CitusTableCacheEntry(Oid distributedRelationId);
|
||||
extern CitusTableCacheEntry * LookupCitusTableCacheEntry(Oid distributedRelationId);
|
||||
extern DistObjectCacheEntry * LookupDistObjectCacheEntry(Oid classid, Oid objid, int32
|
||||
objsubid);
|
||||
extern int32 GetLocalGroupId(void);
|
||||
|
@ -152,7 +152,7 @@ extern bool CheckCitusVersion(int elevel);
|
|||
extern bool CheckAvailableVersion(int elevel);
|
||||
extern bool InstalledAndAvailableVersionsSame(void);
|
||||
extern bool MajorVersionsCompatible(char *leftVersion, char *rightVersion);
|
||||
extern void ErrorIfInconsistentShardIntervals(DistTableCacheEntry *cacheEntry);
|
||||
extern void ErrorIfInconsistentShardIntervals(CitusTableCacheEntry *cacheEntry);
|
||||
extern void EnsureModificationsCanRun(void);
|
||||
extern char LookupDistributionMethod(Oid distributionMethodOid);
|
||||
|
||||
|
|
|
@ -34,7 +34,7 @@ extern bool ShouldSyncTableMetadata(Oid relationId);
|
|||
extern List * MetadataCreateCommands(void);
|
||||
extern List * GetDistributedTableDDLEvents(Oid relationId);
|
||||
extern List * MetadataDropCommands(void);
|
||||
extern char * DistributionCreateCommand(DistTableCacheEntry *cacheEntry);
|
||||
extern char * DistributionCreateCommand(CitusTableCacheEntry *cacheEntry);
|
||||
extern char * DistributionDeleteCommand(const char *schemaName,
|
||||
const char *tableName);
|
||||
extern char * TableOwnerResetCommand(Oid distributedRelationId);
|
||||
|
|
|
@ -48,8 +48,8 @@ extern int CompareRelationShards(const void *leftElement,
|
|||
const void *rightElement);
|
||||
extern int ShardIndex(ShardInterval *shardInterval);
|
||||
extern ShardInterval * FindShardInterval(Datum partitionColumnValue,
|
||||
DistTableCacheEntry *cacheEntry);
|
||||
extern int FindShardIntervalIndex(Datum searchedValue, DistTableCacheEntry *cacheEntry);
|
||||
CitusTableCacheEntry *cacheEntry);
|
||||
extern int FindShardIntervalIndex(Datum searchedValue, CitusTableCacheEntry *cacheEntry);
|
||||
extern int SearchCachedShardInterval(Datum partitionColumnValue,
|
||||
ShardInterval **shardIntervalCache,
|
||||
int shardCount, Oid shardIntervalCollation,
|
||||
|
|
Loading…
Reference in New Issue